Merge branch 'master' into something-strange-with-the-test-about-refreshable-materialized-views

This commit is contained in:
Alexey Milovidov 2024-07-22 16:21:53 +02:00
commit 7f741dd496
71 changed files with 2626 additions and 482 deletions

View File

@ -62,8 +62,8 @@ runs:
if: ${{ inputs.type == 'patch' }} if: ${{ inputs.type == 'patch' }}
shell: bash shell: bash
run: | run: |
python3 ./tests/ci/create_release.py --set-progress-started --progress "update ChangeLog" git checkout master
[ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security"
echo "List versions" echo "List versions"
./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv
echo "Update docker version" echo "Update docker version"
@ -96,17 +96,13 @@ runs:
Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }}
### Changelog category (leave one): ### Changelog category (leave one):
- Not for changelog (changelog entry is not required) - Not for changelog (changelog entry is not required)
- name: Reset changes if Dry-run - name: Complete previous steps and Restore git state
if: ${{ inputs.dry-run }} if: ${{ inputs.type == 'patch' }}
shell: bash shell: bash
run: | run: |
git reset --hard HEAD
- name: Checkout back to GITHUB_REF
shell: bash
run: |
git checkout "$GITHUB_REF_NAME"
# set current progress to OK
python3 ./tests/ci/create_release.py --set-progress-completed python3 ./tests/ci/create_release.py --set-progress-completed
git reset --hard HEAD
git checkout "$GITHUB_REF_NAME"
- name: Create GH Release - name: Create GH Release
shell: bash shell: bash
if: ${{ inputs.type == 'patch' }} if: ${{ inputs.type == 'patch' }}
@ -146,24 +142,23 @@ runs:
if: ${{ inputs.type == 'patch' }} if: ${{ inputs.type == 'patch' }}
shell: bash shell: bash
run: | run: |
python3 ./tests/ci/create_release.py --set-progress-started --progress "docker server release"
cd "./tests/ci" cd "./tests/ci"
python3 ./create_release.py --set-progress-started --progress "docker server release"
export CHECK_NAME="Docker server image" export CHECK_NAME="Docker server image"
python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }}
python3 ./tests/ci/create_release.py --set-progress-completed python3 ./create_release.py --set-progress-completed
- name: Docker clickhouse/clickhouse-keeper building - name: Docker clickhouse/clickhouse-keeper building
if: ${{ inputs.type == 'patch' }} if: ${{ inputs.type == 'patch' }}
shell: bash shell: bash
run: | run: |
python3 ./tests/ci/create_release.py --set-progress-started --progress "docker keeper release"
cd "./tests/ci" cd "./tests/ci"
python3 ./create_release.py --set-progress-started --progress "docker keeper release"
export CHECK_NAME="Docker keeper image" export CHECK_NAME="Docker keeper image"
python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }}
python3 ./tests/ci/create_release.py --set-progress-completed python3 ./create_release.py --set-progress-completed
- name: Set Release progress completed - name: Set current Release progress to Completed with OK
shell: bash shell: bash
run: | run: |
# If we here - set completed status, to post proper Slack OK or FAIL message in the next step
python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" python3 ./tests/ci/create_release.py --set-progress-started --progress "completed"
python3 ./tests/ci/create_release.py --set-progress-completed python3 ./tests/ci/create_release.py --set-progress-completed
- name: Post Slack Message - name: Post Slack Message

View File

@ -297,7 +297,7 @@ Algorithm requires the special column in tables. This column:
- Must contain an ordered sequence. - Must contain an ordered sequence.
- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). - Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md).
- Cant be the only column in the `JOIN` clause. - For `hash` join algorithm it cant be the only column in the `JOIN` clause.
Syntax `ASOF JOIN ... ON`: Syntax `ASOF JOIN ... ON`:
@ -337,7 +337,8 @@ For example, consider the following tables:
`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` cant be joined. `ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` cant be joined.
:::note :::note
`ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. `ASOF JOIN` is supported only by `hash` and `full_sorting_merge` join algorithms.
It's **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine.
::: :::
## PASTE JOIN Usage ## PASTE JOIN Usage

View File

@ -68,6 +68,41 @@ QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes)
return nullptr; return nullptr;
} }
/// Checks if the node is combination of isNull and notEquals functions of two the same arguments
bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, QueryTreeNodePtr & rhs)
{
QueryTreeNodePtrWithHashSet all_arguments;
for (const auto & node : nodes)
{
const auto * func_node = node->as<FunctionNode>();
if (!func_node)
return false;
const auto & arguments = func_node->getArguments().getNodes();
if (func_node->getFunctionName() == "isNull" && arguments.size() == 1)
all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0]));
else if (func_node->getFunctionName() == "notEquals" && arguments.size() == 2)
{
if (arguments[0]->isEqual(*arguments[1]))
return false;
all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0]));
all_arguments.insert(QueryTreeNodePtrWithHash(arguments[1]));
}
else
return false;
if (all_arguments.size() > 2)
return false;
}
if (all_arguments.size() != 2)
return false;
lhs = all_arguments.begin()->node;
rhs = std::next(all_arguments.begin())->node;
return true;
}
bool isBooleanConstant(const QueryTreeNodePtr & node, bool expected_value) bool isBooleanConstant(const QueryTreeNodePtr & node, bool expected_value)
{ {
const auto * constant_node = node->as<ConstantNode>(); const auto * constant_node = node->as<ConstantNode>();
@ -213,11 +248,14 @@ private:
else if (func_name == "and") else if (func_name == "and")
{ {
const auto & and_arguments = argument_function->getArguments().getNodes(); const auto & and_arguments = argument_function->getArguments().getNodes();
bool all_are_is_null = and_arguments.size() == 2 && isNodeFunction(and_arguments[0], "isNull") && isNodeFunction(and_arguments[1], "isNull");
if (all_are_is_null) QueryTreeNodePtr is_null_lhs_arg;
QueryTreeNodePtr is_null_rhs_arg;
if (matchIsNullOfTwoArgs(and_arguments, is_null_lhs_arg, is_null_rhs_arg))
{ {
is_null_argument_to_indices[getFunctionArgument(and_arguments.front(), 0)].push_back(or_operands.size() - 1); is_null_argument_to_indices[is_null_lhs_arg].push_back(or_operands.size() - 1);
is_null_argument_to_indices[getFunctionArgument(and_arguments.back(), 0)].push_back(or_operands.size() - 1); is_null_argument_to_indices[is_null_rhs_arg].push_back(or_operands.size() - 1);
continue;
} }
/// Expression `a = b AND (a IS NOT NULL) AND true AND (b IS NOT NULL)` we can be replaced with `a = b` /// Expression `a = b AND (a IS NOT NULL) AND true AND (b IS NOT NULL)` we can be replaced with `a = b`

View File

@ -206,7 +206,7 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
} }
else else
{ {
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size);
ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed);
} }
} }
@ -239,7 +239,7 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
void * new_buf = ::realloc(buf, new_size); void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf) if (nullptr == new_buf)
{ {
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size);
throw DB::ErrnoException( throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Allocator: Cannot realloc from {} to {}", "Allocator: Cannot realloc from {} to {}",

View File

@ -234,10 +234,10 @@
M(PartsCommitted, "Deprecated. See PartsActive.") \ M(PartsCommitted, "Deprecated. See PartsActive.") \
M(PartsPreActive, "The part is in data_parts, but not used for SELECTs.") \ M(PartsPreActive, "The part is in data_parts, but not used for SELECTs.") \
M(PartsActive, "Active data part, used by current and upcoming SELECTs.") \ M(PartsActive, "Active data part, used by current and upcoming SELECTs.") \
M(AttachedDatabase, "Active database, used by current and upcoming SELECTs.") \ M(AttachedDatabase, "Active databases.") \
M(AttachedTable, "Active table, used by current and upcoming SELECTs.") \ M(AttachedTable, "Active tables.") \
M(AttachedView, "Active view, used by current and upcoming SELECTs.") \ M(AttachedView, "Active views.") \
M(AttachedDictionary, "Active dictionary, used by current and upcoming SELECTs.") \ M(AttachedDictionary, "Active dictionaries.") \
M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \ M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \
M(PartsDeleting, "Not active data part with identity refcounter, it is deleting right now by a cleaner.") \ M(PartsDeleting, "Not active data part with identity refcounter, it is deleting right now by a cleaner.") \
M(PartsDeleteOnDestroy, "Part was moved to another disk and should be deleted in own destructor.") \ M(PartsDeleteOnDestroy, "Part was moved to another disk and should be deleted in own destructor.") \

View File

@ -6,12 +6,17 @@ namespace DB
{ {
String getRandomASCIIString(size_t length) String getRandomASCIIString(size_t length)
{
return getRandomASCIIString(length, thread_local_rng);
}
String getRandomASCIIString(size_t length, pcg64 & rng)
{ {
std::uniform_int_distribution<int> distribution('a', 'z'); std::uniform_int_distribution<int> distribution('a', 'z');
String res; String res;
res.resize(length); res.resize(length);
for (auto & c : res) for (auto & c : res)
c = distribution(thread_local_rng); c = distribution(rng);
return res; return res;
} }

View File

@ -2,11 +2,14 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <pcg_random.hpp>
namespace DB namespace DB
{ {
/// Slow random string. Useful for random names and things like this. Not for generating data. /// Slow random string. Useful for random names and things like this. Not for generating data.
String getRandomASCIIString(size_t length); String getRandomASCIIString(size_t length);
String getRandomASCIIString(size_t length, pcg64 & rng);
} }

View File

@ -195,7 +195,7 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n
snapshot_detached_tables.erase(table_name); snapshot_detached_tables.erase(table_name);
} }
CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); CurrentMetrics::add(CurrentMetrics::AttachedTable);
} }
StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name) StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name)
@ -221,7 +221,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
.metadata_path = getObjectMetadataPath(table_name), .metadata_path = getObjectMetadataPath(table_name),
.is_permanently = false}); .is_permanently = false});
CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); CurrentMetrics::sub(CurrentMetrics::AttachedTable);
} }
return res; return res;
} }

View File

@ -289,8 +289,8 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
tables.erase(it); tables.erase(it);
table_storage->is_detached = true; table_storage->is_detached = true;
if (table_storage->isSystemStorage() == false) if (!table_storage->isSystemStorage() && database_name != DatabaseCatalog::SYSTEM_DATABASE)
CurrentMetrics::sub(getAttachedCounterForStorage(table_storage), 1); CurrentMetrics::sub(getAttachedCounterForStorage(table_storage));
auto table_id = table_storage->getStorageID(); auto table_id = table_storage->getStorageID();
if (table_id.hasUUID()) if (table_id.hasUUID())
@ -334,8 +334,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
/// non-Atomic database the is_detached is set to true before RENAME. /// non-Atomic database the is_detached is set to true before RENAME.
table->is_detached = false; table->is_detached = false;
if (table->isSystemStorage() == false && table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) if (!table->isSystemStorage() && table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE)
CurrentMetrics::add(getAttachedCounterForStorage(table), 1); CurrentMetrics::add(getAttachedCounterForStorage(table));
} }
void DatabaseWithOwnTablesBase::shutdown() void DatabaseWithOwnTablesBase::shutdown()

View File

@ -16,7 +16,6 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <base/hex.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Core/SettingsEnums.h> #include <Core/SettingsEnums.h>
@ -82,13 +81,13 @@
#include <Interpreters/ApplyWithSubqueryVisitor.h> #include <Interpreters/ApplyWithSubqueryVisitor.h>
#include <TableFunctions/TableFunctionFactory.h> #include <TableFunctions/TableFunctionFactory.h>
#include <DataTypes/DataTypeFixedString.h>
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h> #include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h> #include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h> #include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Parsers/QueryParameterVisitor.h> #include <Parsers/QueryParameterVisitor.h>
namespace CurrentMetrics namespace CurrentMetrics
{ {
extern const Metric AttachedTable; extern const Metric AttachedTable;
@ -147,27 +146,27 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
} }
auto db_num_limit = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw; auto db_num_limit = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw;
if (db_num_limit > 0) if (db_num_limit > 0 && !internal)
{ {
size_t db_count = DatabaseCatalog::instance().getDatabases().size(); size_t db_count = DatabaseCatalog::instance().getDatabases().size();
std::vector<String> system_databases = { std::initializer_list<std::string_view> system_databases =
{
DatabaseCatalog::TEMPORARY_DATABASE, DatabaseCatalog::TEMPORARY_DATABASE,
DatabaseCatalog::SYSTEM_DATABASE, DatabaseCatalog::SYSTEM_DATABASE,
DatabaseCatalog::INFORMATION_SCHEMA, DatabaseCatalog::INFORMATION_SCHEMA,
DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE,
DatabaseCatalog::DEFAULT_DATABASE
}; };
for (const auto & system_database : system_databases) for (const auto & system_database : system_databases)
{ {
if (db_count > 0 && DatabaseCatalog::instance().isDatabaseExist(system_database)) if (db_count > 0 && DatabaseCatalog::instance().isDatabaseExist(std::string(system_database)))
db_count--; --db_count;
} }
if (db_count >= db_num_limit) if (db_count >= db_num_limit)
throw Exception(ErrorCodes::TOO_MANY_DATABASES, throw Exception(ErrorCodes::TOO_MANY_DATABASES,
"Too many databases in the Clickhouse. " "Too many databases. "
"The limit (setting 'max_database_num_to_throw') is set to {}, current number of databases is {}", "The limit (server configuration parameter `max_database_num_to_throw`) is set to {}, the current number of databases is {}",
db_num_limit, db_count); db_num_limit, db_count);
} }
@ -1601,13 +1600,13 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
} }
UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw; UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw;
if (table_num_limit > 0 && create.getDatabase() != DatabaseCatalog::SYSTEM_DATABASE) if (table_num_limit > 0 && !internal)
{ {
UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable); UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable);
if (table_count >= table_num_limit) if (table_count >= table_num_limit)
throw Exception(ErrorCodes::TOO_MANY_TABLES, throw Exception(ErrorCodes::TOO_MANY_TABLES,
"Too many tables in the Clickhouse. " "Too many tables. "
"The limit (setting 'max_table_num_to_throw') is set to {}, current number of tables is {}", "The limit (server configuration parameter `max_table_num_to_throw`) is set to {}, the current number of tables is {}",
table_num_limit, table_count); table_num_limit, table_count);
} }

View File

@ -1726,7 +1726,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
const auto & join_clause = table_join.getOnlyClause(); const auto & join_clause = table_join.getOnlyClause();
auto join_kind = table_join.kind(); auto join_kind = table_join.kind();
bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); auto join_strictness = table_join.strictness();
bool join_type_allows_filtering = (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Any)
&& (isInner(join_kind) || isLeft(join_kind) || isRight(join_kind));
auto has_non_const = [](const Block & block, const auto & keys) auto has_non_const = [](const Block & block, const auto & keys)
{ {
@ -1745,7 +1748,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
bool has_non_const_keys = has_non_const(query_plan.getCurrentDataStream().header, join_clause.key_names_left) bool has_non_const_keys = has_non_const(query_plan.getCurrentDataStream().header, join_clause.key_names_left)
&& has_non_const(joined_plan->getCurrentDataStream().header, join_clause.key_names_right); && has_non_const(joined_plan->getCurrentDataStream().header, join_clause.key_names_right);
if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering && has_non_const_keys) if (settings.max_rows_in_set_to_optimize_join > 0 && join_type_allows_filtering && has_non_const_keys)
{ {
auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left);
auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right);

View File

@ -2743,7 +2743,7 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
/// 'AND' can be both boolean function and part of the '... BETWEEN ... AND ...' operator /// 'AND' can be both boolean function and part of the '... BETWEEN ... AND ...' operator
if (op.function_name == "and" && layers.back()->between_counter) if (op.function_name == "and" && layers.back()->between_counter)
{ {
layers.back()->between_counter--; --layers.back()->between_counter;
op = finish_between_operator; op = finish_between_operator;
} }

View File

@ -77,7 +77,6 @@ namespace ErrorCodes
extern const int INVALID_JOIN_ON_EXPRESSION; extern const int INVALID_JOIN_ON_EXPRESSION;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int SYNTAX_ERROR;
extern const int ACCESS_DENIED; extern const int ACCESS_DENIED;
extern const int PARAMETER_OUT_OF_BOUND; extern const int PARAMETER_OUT_OF_BOUND;
extern const int TOO_MANY_COLUMNS; extern const int TOO_MANY_COLUMNS;
@ -1397,12 +1396,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
{ {
if (!join_clause.hasASOF()) if (!join_clause.hasASOF())
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} no inequality in ASOF JOIN ON section.", "JOIN {} no inequality in ASOF JOIN ON section",
join_node.formatASTForErrorMessage());
if (table_join_clause.key_names_left.size() <= 1)
throw Exception(ErrorCodes::SYNTAX_ERROR,
"JOIN {} ASOF join needs at least one equi-join column",
join_node.formatASTForErrorMessage()); join_node.formatASTForErrorMessage());
} }
@ -1524,7 +1518,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
{ {
const auto & join_clause = table_join->getOnlyClause(); const auto & join_clause = table_join->getOnlyClause();
bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); bool join_type_allows_filtering = (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Any)
&& (isInner(join_kind) || isLeft(join_kind) || isRight(join_kind));
auto has_non_const = [](const Block & block, const auto & keys) auto has_non_const = [](const Block & block, const auto & keys)
{ {
@ -1544,7 +1540,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
bool has_non_const_keys = has_non_const(left_plan.getCurrentDataStream().header, join_clause.key_names_left) bool has_non_const_keys = has_non_const(left_plan.getCurrentDataStream().header, join_clause.key_names_left)
&& has_non_const(right_plan.getCurrentDataStream().header, join_clause.key_names_right); && has_non_const(right_plan.getCurrentDataStream().header, join_clause.key_names_right);
if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering && has_non_const_keys) if (settings.max_rows_in_set_to_optimize_join > 0 && join_type_allows_filtering && has_non_const_keys)
{ {
auto * left_set = add_create_set(left_plan, join_clause.key_names_left, JoinTableSide::Left); auto * left_set = add_create_set(left_plan, join_clause.key_names_left, JoinTableSide::Left);
auto * right_set = add_create_set(right_plan, join_clause.key_names_right, JoinTableSide::Right); auto * right_set = add_create_set(right_plan, join_clause.key_names_right, JoinTableSide::Right);

View File

@ -34,13 +34,20 @@ namespace ErrorCodes
namespace namespace
{ {
FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns) FullMergeJoinCursorPtr createCursor(const Block & block, const Names & columns, JoinStrictness strictness)
{ {
SortDescription desc; SortDescription desc;
desc.reserve(columns.size()); desc.reserve(columns.size());
for (const auto & name : columns) for (const auto & name : columns)
desc.emplace_back(name); desc.emplace_back(name);
return std::make_unique<FullMergeJoinCursor>(block, desc); return std::make_unique<FullMergeJoinCursor>(block, desc, strictness == JoinStrictness::Asof);
}
bool ALWAYS_INLINE isNullAt(const IColumn & column, size_t row)
{
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&column))
return nullable_column->isNullAt(row);
return false;
} }
template <bool has_left_nulls, bool has_right_nulls> template <bool has_left_nulls, bool has_right_nulls>
@ -54,7 +61,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if (left_nullable && right_nullable) if (left_nullable && right_nullable)
{ {
int res = left_nullable->compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); int res = left_nullable->compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint);
if (res) if (res != 0)
return res; return res;
/// NULL != NULL case /// NULL != NULL case
@ -90,9 +97,10 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos, int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos,
const SortCursorImpl & rhs, size_t rpos, const SortCursorImpl & rhs, size_t rpos,
size_t key_length,
int null_direction_hint) int null_direction_hint)
{ {
for (size_t i = 0; i < lhs.sort_columns_size; ++i) for (size_t i = 0; i < key_length; ++i)
{ {
/// TODO(@vdimir): use nullableCompareAt only if there's nullable columns /// TODO(@vdimir): use nullableCompareAt only if there's nullable columns
int cmp = nullableCompareAt<true, true>(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos, null_direction_hint); int cmp = nullableCompareAt<true, true>(*lhs.sort_columns[i], *rhs.sort_columns[i], lpos, rpos, null_direction_hint);
@ -104,13 +112,18 @@ int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, size_t lpos,
int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs, int null_direction_hint) int ALWAYS_INLINE compareCursors(const SortCursorImpl & lhs, const SortCursorImpl & rhs, int null_direction_hint)
{ {
return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), null_direction_hint); return compareCursors(lhs, lhs.getRow(), rhs, rhs.getRow(), lhs.sort_columns_size, null_direction_hint);
}
int compareAsofCursors(const FullMergeJoinCursor & lhs, const FullMergeJoinCursor & rhs, int null_direction_hint)
{
return nullableCompareAt<true, true>(*lhs.getAsofColumn(), *rhs.getAsofColumn(), lhs->getRow(), rhs->getRow(), null_direction_hint);
} }
bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint) bool ALWAYS_INLINE totallyLess(SortCursorImpl & lhs, SortCursorImpl & rhs, int null_direction_hint)
{ {
/// The last row of left cursor is less than the current row of the right cursor. /// The last row of left cursor is less than the current row of the right cursor.
int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow(), null_direction_hint); int cmp = compareCursors(lhs, lhs.rows - 1, rhs, rhs.getRow(), lhs.sort_columns_size, null_direction_hint);
return cmp < 0; return cmp < 0;
} }
@ -222,25 +235,136 @@ Chunk getRowFromChunk(const Chunk & chunk, size_t pos)
return result; return result;
} }
void inline addRange(PaddedPODArray<UInt64> & left_map, size_t start, size_t end) void inline addRange(PaddedPODArray<UInt64> & values, UInt64 start, UInt64 end)
{ {
assert(end > start); assert(end > start);
for (size_t i = start; i < end; ++i) for (UInt64 i = start; i < end; ++i)
left_map.push_back(i); values.push_back(i);
} }
void inline addMany(PaddedPODArray<UInt64> & left_or_right_map, size_t idx, size_t num) void inline addMany(PaddedPODArray<UInt64> & values, UInt64 value, size_t num)
{ {
for (size_t i = 0; i < num; ++i) values.resize_fill(values.size() + num, value);
left_or_right_map.push_back(idx);
} }
} }
FullMergeJoinCursor::FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_) JoinKeyRow::JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos)
: sample_block(materializeBlock(sample_block_).cloneEmpty()), desc(description_)
{ {
row.reserve(cursor->sort_columns.size());
for (const auto & col : cursor->sort_columns)
{
auto new_col = col->cloneEmpty();
new_col->insertFrom(*col, pos);
row.push_back(std::move(new_col));
}
if (const IColumn * asof_column = cursor.getAsofColumn())
{
if (const auto * nullable_asof_column = checkAndGetColumn<ColumnNullable>(asof_column))
{
/// We save matched column, and since NULL do not match anything, we can't use it as a key
chassert(!nullable_asof_column->isNullAt(pos));
asof_column = nullable_asof_column->getNestedColumnPtr().get();
}
auto new_col = asof_column->cloneEmpty();
new_col->insertFrom(*asof_column, pos);
row.push_back(std::move(new_col));
}
} }
void JoinKeyRow::reset()
{
row.clear();
}
bool JoinKeyRow::equals(const FullMergeJoinCursor & cursor) const
{
if (row.empty())
return false;
for (size_t i = 0; i < cursor->sort_columns_size; ++i)
{
// int cmp = this->row[i]->compareAt(0, cursor->getRow(), *(cursor->sort_columns[i]), cursor->desc[i].nulls_direction);
int cmp = nullableCompareAt<true, true>(*this->row[i], *cursor->sort_columns[i], 0, cursor->getRow(), cursor->desc[i].nulls_direction);
if (cmp != 0)
return false;
}
return true;
}
bool JoinKeyRow::asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const
{
chassert(this->row.size() == cursor->sort_columns_size + 1);
if (!equals(cursor))
return false;
const auto & asof_row = row.back();
if (isNullAt(*asof_row, 0) || isNullAt(*cursor.getAsofColumn(), cursor->getRow()))
return false;
int cmp = 0;
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(cursor.getAsofColumn()))
cmp = nullable_column->getNestedColumn().compareAt(cursor->getRow(), 0, *asof_row, 1);
else
cmp = cursor.getAsofColumn()->compareAt(cursor->getRow(), 0, *asof_row, 1);
return (asof_inequality == ASOFJoinInequality::Less && cmp < 0)
|| (asof_inequality == ASOFJoinInequality::LessOrEquals && cmp <= 0)
|| (asof_inequality == ASOFJoinInequality::Greater && cmp > 0)
|| (asof_inequality == ASOFJoinInequality::GreaterOrEquals && cmp >= 0);
}
void AnyJoinState::set(size_t source_num, const FullMergeJoinCursor & cursor)
{
assert(cursor->rows);
keys[source_num] = JoinKeyRow(cursor, cursor->rows - 1);
}
void AnyJoinState::reset(size_t source_num)
{
keys[source_num].reset();
value.clear();
}
void AnyJoinState::setValue(Chunk value_)
{
value = std::move(value_);
}
bool AnyJoinState::empty() const { return keys[0].row.empty() && keys[1].row.empty(); }
void AsofJoinState::set(const FullMergeJoinCursor & rcursor, size_t rpos)
{
key = JoinKeyRow(rcursor, rpos);
value = rcursor.getCurrent().clone();
value_row = rpos;
}
void AsofJoinState::reset()
{
key.reset();
value.clear();
}
FullMergeJoinCursor::FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_, bool is_asof)
: sample_block(materializeBlock(sample_block_).cloneEmpty())
, desc(description_)
{
if (desc.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty sort description for FullMergeJoinCursor");
if (is_asof)
{
/// For ASOF join prefix of sort description is used for equality comparison
/// and the last column is used for inequality comparison and is handled separately
auto asof_column_description = desc.back();
desc.pop_back();
chassert(asof_column_description.direction == 1 && asof_column_description.nulls_direction == 1);
asof_column_position = sample_block.getPositionByName(asof_column_description.column_name);
}
}
const Chunk & FullMergeJoinCursor::getCurrent() const const Chunk & FullMergeJoinCursor::getCurrent() const
{ {
@ -278,48 +402,103 @@ bool FullMergeJoinCursor::fullyCompleted() const
return !cursor.isValid() && recieved_all_blocks; return !cursor.isValid() && recieved_all_blocks;
} }
String FullMergeJoinCursor::dump() const
{
Strings row_dump;
if (cursor.isValid())
{
Field val;
for (size_t i = 0; i < cursor.sort_columns_size; ++i)
{
cursor.sort_columns[i]->get(cursor.getRow(), val);
row_dump.push_back(val.dump());
}
if (const auto * asof_column = getAsofColumn())
{
asof_column->get(cursor.getRow(), val);
row_dump.push_back(val.dump());
}
}
return fmt::format("<{}/{}{}>[{}]",
cursor.getRow(), cursor.rows,
recieved_all_blocks ? "(finished)" : "",
fmt::join(row_dump, ", "));
}
MergeJoinAlgorithm::MergeJoinAlgorithm( MergeJoinAlgorithm::MergeJoinAlgorithm(
JoinPtr table_join_, JoinKind kind_,
JoinStrictness strictness_,
const TableJoin::JoinOnClause & on_clause_,
const Blocks & input_headers, const Blocks & input_headers,
size_t max_block_size_) size_t max_block_size_)
: table_join(table_join_) : kind(kind_)
, strictness(strictness_)
, max_block_size(max_block_size_) , max_block_size(max_block_size_)
, log(getLogger("MergeJoinAlgorithm")) , log(getLogger("MergeJoinAlgorithm"))
{ {
if (input_headers.size() != 2) if (input_headers.size() != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinAlgorithm requires exactly two inputs"); throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinAlgorithm requires exactly two inputs");
auto strictness = table_join->getTableJoin().strictness(); if (strictness != JoinStrictness::Any && strictness != JoinStrictness::All && strictness != JoinStrictness::Asof)
if (strictness != JoinStrictness::Any && strictness != JoinStrictness::All)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness {}", strictness); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for strictness {}", strictness);
auto kind = table_join->getTableJoin().kind(); if (strictness == JoinStrictness::Asof)
{
if (kind != JoinKind::Left && kind != JoinKind::Inner)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not implement ASOF {} join", kind);
}
if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind)) if (!isInner(kind) && !isLeft(kind) && !isRight(kind) && !isFull(kind))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for kind {}", kind); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm is not implemented for kind {}", kind);
const auto & join_on = table_join->getTableJoin().getOnlyClause(); if (on_clause_.on_filter_condition_left || on_clause_.on_filter_condition_right)
if (join_on.on_filter_condition_left || join_on.on_filter_condition_right)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoinAlgorithm does not support ON filter conditions");
cursors = { cursors = {
createCursor(input_headers[0], join_on.key_names_left), createCursor(input_headers[0], on_clause_.key_names_left, strictness),
createCursor(input_headers[1], join_on.key_names_right) createCursor(input_headers[1], on_clause_.key_names_right, strictness),
}; };
}
for (const auto & [left_key, right_key] : table_join->getTableJoin().leftToRightKeyRemap()) MergeJoinAlgorithm::MergeJoinAlgorithm(
JoinPtr join_ptr,
const Blocks & input_headers,
size_t max_block_size_)
: MergeJoinAlgorithm(
join_ptr->getTableJoin().kind(),
join_ptr->getTableJoin().strictness(),
join_ptr->getTableJoin().getOnlyClause(),
input_headers,
max_block_size_)
{
for (const auto & [left_key, right_key] : join_ptr->getTableJoin().leftToRightKeyRemap())
{ {
size_t left_idx = input_headers[0].getPositionByName(left_key); size_t left_idx = input_headers[0].getPositionByName(left_key);
size_t right_idx = input_headers[1].getPositionByName(right_key); size_t right_idx = input_headers[1].getPositionByName(right_key);
left_to_right_key_remap[left_idx] = right_idx; left_to_right_key_remap[left_idx] = right_idx;
} }
const auto *smjPtr = typeid_cast<const FullSortingMergeJoin *>(table_join.get()); const auto *smjPtr = typeid_cast<const FullSortingMergeJoin *>(join_ptr.get());
if (smjPtr) if (smjPtr)
{ {
null_direction_hint = smjPtr->getNullDirection(); null_direction_hint = smjPtr->getNullDirection();
} }
if (strictness == JoinStrictness::Asof)
setAsofInequality(join_ptr->getTableJoin().getAsofInequality());
}
void MergeJoinAlgorithm::setAsofInequality(ASOFJoinInequality asof_inequality_)
{
if (strictness != JoinStrictness::Asof)
throw Exception(ErrorCodes::LOGICAL_ERROR, "setAsofInequality is only supported for ASOF joins");
if (asof_inequality_ == ASOFJoinInequality::None)
throw Exception(ErrorCodes::LOGICAL_ERROR, "ASOF inequality cannot be None");
asof_inequality = asof_inequality_;
} }
void MergeJoinAlgorithm::logElapsed(double seconds) void MergeJoinAlgorithm::logElapsed(double seconds)
@ -407,7 +586,7 @@ struct AllJoinImpl
size_t lnum = nextDistinct(left_cursor.cursor); size_t lnum = nextDistinct(left_cursor.cursor);
size_t rnum = nextDistinct(right_cursor.cursor); size_t rnum = nextDistinct(right_cursor.cursor);
bool all_fit_in_block = std::max(left_map.size(), right_map.size()) + lnum * rnum <= max_block_size; bool all_fit_in_block = !max_block_size || std::max(left_map.size(), right_map.size()) + lnum * rnum <= max_block_size;
bool have_all_ranges = left_cursor.cursor.isValid() && right_cursor.cursor.isValid(); bool have_all_ranges = left_cursor.cursor.isValid() && right_cursor.cursor.isValid();
if (all_fit_in_block && have_all_ranges) if (all_fit_in_block && have_all_ranges)
{ {
@ -421,7 +600,7 @@ struct AllJoinImpl
else else
{ {
assert(state == nullptr); assert(state == nullptr);
state = std::make_unique<AllJoinState>(left_cursor.cursor, lpos, right_cursor.cursor, rpos); state = std::make_unique<AllJoinState>(left_cursor, lpos, right_cursor, rpos);
state->addRange(0, left_cursor.getCurrent().clone(), lpos, lnum); state->addRange(0, left_cursor.getCurrent().clone(), lpos, lnum);
state->addRange(1, right_cursor.getCurrent().clone(), rpos, rnum); state->addRange(1, right_cursor.getCurrent().clone(), rpos, rnum);
return; return;
@ -466,6 +645,17 @@ void dispatchKind(JoinKind kind, Args && ... args)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported join kind: \"{}\"", kind);
} }
MutableColumns MergeJoinAlgorithm::getEmptyResultColumns() const
{
MutableColumns result_cols;
for (size_t i = 0; i < 2; ++i)
{
for (const auto & col : cursors[i]->sampleColumns())
result_cols.push_back(col->cloneEmpty());
}
return result_cols;
}
std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAllJoinState() std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAllJoinState()
{ {
if (all_join_state && all_join_state->finished()) if (all_join_state && all_join_state->finished())
@ -479,7 +669,7 @@ std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAllJoinState
/// Accumulate blocks with same key in all_join_state /// Accumulate blocks with same key in all_join_state
for (size_t i = 0; i < 2; ++i) for (size_t i = 0; i < 2; ++i)
{ {
if (cursors[i]->cursor.isValid() && all_join_state->keys[i].equals(cursors[i]->cursor)) if (cursors[i]->cursor.isValid() && all_join_state->keys[i].equals(*cursors[i]))
{ {
size_t pos = cursors[i]->cursor.getRow(); size_t pos = cursors[i]->cursor.getRow();
size_t num = nextDistinct(cursors[i]->cursor); size_t num = nextDistinct(cursors[i]->cursor);
@ -499,15 +689,10 @@ std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAllJoinState
stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, all_join_state->blocksStored()); stat.max_blocks_loaded = std::max(stat.max_blocks_loaded, all_join_state->blocksStored());
/// join all rows with current key /// join all rows with current key
MutableColumns result_cols; MutableColumns result_cols = getEmptyResultColumns();
for (size_t i = 0; i < 2; ++i)
{
for (const auto & col : cursors[i]->sampleColumns())
result_cols.push_back(col->cloneEmpty());
}
size_t total_rows = 0; size_t total_rows = 0;
while (total_rows < max_block_size) while (!max_block_size || total_rows < max_block_size)
{ {
const auto & left_range = all_join_state->getLeft(); const auto & left_range = all_join_state->getLeft();
const auto & right_range = all_join_state->getRight(); const auto & right_range = all_join_state->getRight();
@ -532,7 +717,52 @@ std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAllJoinState
return {}; return {};
} }
MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAsofJoinState()
{
if (strictness != JoinStrictness::Asof)
return {};
if (!cursors[1]->fullyCompleted())
return {};
auto & left_cursor = *cursors[0];
const auto & left_columns = left_cursor.getCurrent().getColumns();
MutableColumns result_cols = getEmptyResultColumns();
while (left_cursor->isValid() && asof_join_state.hasMatch(left_cursor, asof_inequality))
{
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, left_cursor->getRow());
for (const auto & col : asof_join_state.value.getColumns())
result_cols[i++]->insertFrom(*col, asof_join_state.value_row);
chassert(i == result_cols.size());
left_cursor->next();
}
while (isLeft(kind) && left_cursor->isValid())
{
/// return row with default values at right side
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, left_cursor->getRow());
for (; i < result_cols.size(); ++i)
result_cols[i]->insertDefault();
chassert(i == result_cols.size());
left_cursor->next();
}
size_t result_rows = result_cols.empty() ? 0 : result_cols.front()->size();
if (result_rows)
return Status(Chunk(std::move(result_cols), result_rows));
return {};
}
MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin()
{ {
PaddedPODArray<UInt64> idx_map[2]; PaddedPODArray<UInt64> idx_map[2];
@ -595,7 +825,7 @@ struct AnyJoinImpl
FullMergeJoinCursor & right_cursor, FullMergeJoinCursor & right_cursor,
PaddedPODArray<UInt64> & left_map, PaddedPODArray<UInt64> & left_map,
PaddedPODArray<UInt64> & right_map, PaddedPODArray<UInt64> & right_map,
AnyJoinState & state, AnyJoinState & any_join_state,
int null_direction_hint) int null_direction_hint)
{ {
assert(enabled); assert(enabled);
@ -656,21 +886,21 @@ struct AnyJoinImpl
} }
} }
/// Remember index of last joined row to propagate it to next block /// Remember last joined row to propagate it to next block
state.setValue({}); any_join_state.setValue({});
if (!left_cursor->isValid()) if (!left_cursor->isValid())
{ {
state.set(0, left_cursor.cursor); any_join_state.set(0, left_cursor);
if (cmp == 0 && isLeft(kind)) if (cmp == 0 && isLeft(kind))
state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos)); any_join_state.setValue(getRowFromChunk(right_cursor.getCurrent(), rpos));
} }
if (!right_cursor->isValid()) if (!right_cursor->isValid())
{ {
state.set(1, right_cursor.cursor); any_join_state.set(1, right_cursor);
if (cmp == 0 && isRight(kind)) if (cmp == 0 && isRight(kind))
state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos)); any_join_state.setValue(getRowFromChunk(left_cursor.getCurrent(), lpos));
} }
} }
}; };
@ -680,40 +910,34 @@ std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAnyJoinState
if (any_join_state.empty()) if (any_join_state.empty())
return {}; return {};
auto kind = table_join->getTableJoin().kind();
Chunk result; Chunk result;
for (size_t source_num = 0; source_num < 2; ++source_num) for (size_t source_num = 0; source_num < 2; ++source_num)
{ {
auto & current = *cursors[source_num]; auto & current = *cursors[source_num];
auto & state = any_join_state; if (any_join_state.keys[source_num].equals(current))
if (any_join_state.keys[source_num].equals(current.cursor))
{ {
size_t start_pos = current->getRow(); size_t start_pos = current->getRow();
size_t length = nextDistinct(current.cursor); size_t length = nextDistinct(current.cursor);
if (length && isLeft(kind) && source_num == 0) if (length && isLeft(kind) && source_num == 0)
{ {
if (state.value) if (any_join_state.value)
result = copyChunkResized(current.getCurrent(), state.value, start_pos, length); result = copyChunkResized(current.getCurrent(), any_join_state.value, start_pos, length);
else else
result = createBlockWithDefaults(source_num, start_pos, length); result = createBlockWithDefaults(source_num, start_pos, length);
} }
if (length && isRight(kind) && source_num == 1) if (length && isRight(kind) && source_num == 1)
{ {
if (state.value) if (any_join_state.value)
result = copyChunkResized(state.value, current.getCurrent(), start_pos, length); result = copyChunkResized(any_join_state.value, current.getCurrent(), start_pos, length);
else else
result = createBlockWithDefaults(source_num, start_pos, length); result = createBlockWithDefaults(source_num, start_pos, length);
} }
/// We've found row with other key, no need to skip more rows with current key
if (current->isValid()) if (current->isValid())
{ any_join_state.keys[source_num].reset();
state.keys[source_num].reset();
}
} }
else else
{ {
@ -726,7 +950,7 @@ std::optional<MergeJoinAlgorithm::Status> MergeJoinAlgorithm::handleAnyJoinState
return {}; return {};
} }
MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind) MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin()
{ {
if (auto result = handleAnyJoinState()) if (auto result = handleAnyJoinState())
return std::move(*result); return std::move(*result);
@ -771,10 +995,151 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::anyJoin(JoinKind kind)
return Status(std::move(result)); return Status(std::move(result));
} }
MergeJoinAlgorithm::Status MergeJoinAlgorithm::asofJoin()
{
auto & left_cursor = *cursors[0];
if (!left_cursor->isValid())
return Status(0);
auto & right_cursor = *cursors[1];
if (!right_cursor->isValid())
return Status(1);
const auto & left_columns = left_cursor.getCurrent().getColumns();
const auto & right_columns = right_cursor.getCurrent().getColumns();
MutableColumns result_cols = getEmptyResultColumns();
while (left_cursor->isValid() && right_cursor->isValid())
{
auto lpos = left_cursor->getRow();
auto rpos = right_cursor->getRow();
auto cmp = compareCursors(*left_cursor, *right_cursor, null_direction_hint);
if (cmp == 0)
{
if (isNullAt(*left_cursor.getAsofColumn(), lpos))
cmp = -1;
if (isNullAt(*right_cursor.getAsofColumn(), rpos))
cmp = 1;
}
if (cmp == 0)
{
auto asof_cmp = compareAsofCursors(left_cursor, right_cursor, null_direction_hint);
if ((asof_inequality == ASOFJoinInequality::Less && asof_cmp <= -1)
|| (asof_inequality == ASOFJoinInequality::LessOrEquals && asof_cmp <= 0))
{
/// First row in right table that is greater (or equal) than current row in left table
/// matches asof join condition the best
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, lpos);
for (const auto & col : right_columns)
result_cols[i++]->insertFrom(*col, rpos);
chassert(i == result_cols.size());
left_cursor->next();
continue;
}
if (asof_inequality == ASOFJoinInequality::Less || asof_inequality == ASOFJoinInequality::LessOrEquals)
{
/// Asof condition is not (yet) satisfied, skip row in right table
right_cursor->next();
continue;
}
if ((asof_inequality == ASOFJoinInequality::Greater && asof_cmp >= 1)
|| (asof_inequality == ASOFJoinInequality::GreaterOrEquals && asof_cmp >= 0))
{
/// condition is satisfied, remember this row and move next to try to find better match
asof_join_state.set(right_cursor, rpos);
right_cursor->next();
continue;
}
if (asof_inequality == ASOFJoinInequality::Greater || asof_inequality == ASOFJoinInequality::GreaterOrEquals)
{
/// Asof condition is not satisfied anymore, use last matched row from right table
if (asof_join_state.hasMatch(left_cursor, asof_inequality))
{
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, lpos);
for (const auto & col : asof_join_state.value.getColumns())
result_cols[i++]->insertFrom(*col, asof_join_state.value_row);
chassert(i == result_cols.size());
}
else
{
asof_join_state.reset();
if (isLeft(kind))
{
/// return row with default values at right side
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, lpos);
for (; i < result_cols.size(); ++i)
result_cols[i]->insertDefault();
chassert(i == result_cols.size());
}
}
left_cursor->next();
continue;
}
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO: implement ASOF equality join");
}
else if (cmp < 0)
{
if (asof_join_state.hasMatch(left_cursor, asof_inequality))
{
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertFrom(*col, lpos);
for (const auto & col : asof_join_state.value.getColumns())
result_cols[i++]->insertFrom(*col, asof_join_state.value_row);
chassert(i == result_cols.size());
left_cursor->next();
continue;
}
else
{
asof_join_state.reset();
}
/// no matches for rows in left table, just pass them through
size_t num = nextDistinct(*left_cursor);
if (isLeft(kind) && num)
{
/// return them with default values at right side
size_t i = 0;
for (const auto & col : left_columns)
result_cols[i++]->insertRangeFrom(*col, lpos, num);
for (; i < result_cols.size(); ++i)
result_cols[i]->insertManyDefaults(num);
chassert(i == result_cols.size());
}
}
else
{
/// skip rows in right table until we find match for current row in left table
nextDistinct(*right_cursor);
}
}
size_t num_rows = result_cols.empty() ? 0 : result_cols.front()->size();
return Status(Chunk(std::move(result_cols), num_rows));
}
/// if `source_num == 0` get data from left cursor and fill defaults at right /// if `source_num == 0` get data from left cursor and fill defaults at right
/// otherwise - vice versa /// otherwise - vice versa
Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const
{ {
ColumnRawPtrs cols; ColumnRawPtrs cols;
{ {
const auto & columns_left = source_num == 0 ? cursors[0]->getCurrent().getColumns() : cursors[0]->sampleColumns(); const auto & columns_left = source_num == 0 ? cursors[0]->getCurrent().getColumns() : cursors[0]->sampleColumns();
@ -797,7 +1162,6 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num, size_t star
cols.push_back(col.get()); cols.push_back(col.get());
} }
} }
Chunk result_chunk; Chunk result_chunk;
copyColumnsResized(cols, start, num_rows, result_chunk); copyColumnsResized(cols, start, num_rows, result_chunk);
return result_chunk; return result_chunk;
@ -813,7 +1177,6 @@ Chunk MergeJoinAlgorithm::createBlockWithDefaults(size_t source_num)
IMergingAlgorithm::Status MergeJoinAlgorithm::merge() IMergingAlgorithm::Status MergeJoinAlgorithm::merge()
{ {
auto kind = table_join->getTableJoin().kind();
if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted()) if (!cursors[0]->cursor.isValid() && !cursors[0]->fullyCompleted())
return Status(0); return Status(0);
@ -821,11 +1184,11 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge()
if (!cursors[1]->cursor.isValid() && !cursors[1]->fullyCompleted()) if (!cursors[1]->cursor.isValid() && !cursors[1]->fullyCompleted())
return Status(1); return Status(1);
if (auto result = handleAllJoinState()) if (auto result = handleAllJoinState())
{
return std::move(*result); return std::move(*result);
}
if (auto result = handleAsofJoinState())
return std::move(*result);
if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted()) if (cursors[0]->fullyCompleted() || cursors[1]->fullyCompleted())
{ {
@ -839,7 +1202,7 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge()
} }
/// check if blocks are not intersecting at all /// check if blocks are not intersecting at all
if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor, null_direction_hint); cmp != 0) if (int cmp = totallyCompare(cursors[0]->cursor, cursors[1]->cursor, null_direction_hint); cmp != 0 && strictness != JoinStrictness::Asof)
{ {
if (cmp < 0) if (cmp < 0)
{ {
@ -858,13 +1221,14 @@ IMergingAlgorithm::Status MergeJoinAlgorithm::merge()
} }
} }
auto strictness = table_join->getTableJoin().strictness();
if (strictness == JoinStrictness::Any) if (strictness == JoinStrictness::Any)
return anyJoin(kind); return anyJoin();
if (strictness == JoinStrictness::All) if (strictness == JoinStrictness::All)
return allJoin(kind); return allJoin();
if (strictness == JoinStrictness::Asof)
return asofJoin();
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported strictness '{}'", strictness); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported strictness '{}'", strictness);
} }
@ -883,9 +1247,26 @@ MergeJoinTransform::MergeJoinTransform(
/* always_read_till_end_= */ false, /* always_read_till_end_= */ false,
/* empty_chunk_on_finish_= */ true, /* empty_chunk_on_finish_= */ true,
table_join, input_headers, max_block_size) table_join, input_headers, max_block_size)
, log(getLogger("MergeJoinTransform"))
{ {
LOG_TRACE(log, "Use MergeJoinTransform"); }
MergeJoinTransform::MergeJoinTransform(
JoinKind kind_,
JoinStrictness strictness_,
const TableJoin::JoinOnClause & on_clause_,
const Blocks & input_headers,
const Block & output_header,
size_t max_block_size,
UInt64 limit_hint_)
: IMergingTransform<MergeJoinAlgorithm>(
input_headers,
output_header,
/* have_all_inputs_= */ true,
limit_hint_,
/* always_read_till_end_= */ false,
/* empty_chunk_on_finish_= */ true,
kind_, strictness_, on_clause_, input_headers, max_block_size)
{
} }
void MergeJoinTransform::onFinish() void MergeJoinTransform::onFinish()

View File

@ -8,6 +8,7 @@
#include <unordered_map> #include <unordered_map>
#include <utility> #include <utility>
#include <vector> #include <vector>
#include <array>
#include <boost/core/noncopyable.hpp> #include <boost/core/noncopyable.hpp>
@ -19,6 +20,7 @@
#include <Processors/Chunk.h> #include <Processors/Chunk.h>
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h> #include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
#include <Processors/Merges/IMergingTransform.h> #include <Processors/Merges/IMergingTransform.h>
#include <Interpreters/TableJoin.h>
namespace Poco { class Logger; } namespace Poco { class Logger; }
@ -35,57 +37,28 @@ using FullMergeJoinCursorPtr = std::unique_ptr<FullMergeJoinCursor>;
/// Used instead of storing previous block /// Used instead of storing previous block
struct JoinKeyRow struct JoinKeyRow
{ {
std::vector<ColumnPtr> row;
JoinKeyRow() = default; JoinKeyRow() = default;
explicit JoinKeyRow(const SortCursorImpl & impl_, size_t pos) JoinKeyRow(const FullMergeJoinCursor & cursor, size_t pos);
{
row.reserve(impl_.sort_columns.size());
for (const auto & col : impl_.sort_columns)
{
auto new_col = col->cloneEmpty();
new_col->insertFrom(*col, pos);
row.push_back(std::move(new_col));
}
}
void reset() bool equals(const FullMergeJoinCursor & cursor) const;
{ bool asofMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const;
row.clear();
}
bool equals(const SortCursorImpl & impl) const void reset();
{
if (row.empty())
return false;
assert(this->row.size() == impl.sort_columns_size); std::vector<ColumnPtr> row;
for (size_t i = 0; i < impl.sort_columns_size; ++i)
{
int cmp = this->row[i]->compareAt(0, impl.getRow(), *impl.sort_columns[i], impl.desc[i].nulls_direction);
if (cmp != 0)
return false;
}
return true;
}
}; };
/// Remembers previous key if it was joined in previous block /// Remembers previous key if it was joined in previous block
class AnyJoinState : boost::noncopyable class AnyJoinState : boost::noncopyable
{ {
public: public:
AnyJoinState() = default; void set(size_t source_num, const FullMergeJoinCursor & cursor);
void setValue(Chunk value_);
void set(size_t source_num, const SortCursorImpl & cursor) void reset(size_t source_num);
{
assert(cursor.rows);
keys[source_num] = JoinKeyRow(cursor, cursor.rows - 1);
}
void setValue(Chunk value_) { value = std::move(value_); } bool empty() const;
bool empty() const { return keys[0].row.empty() && keys[1].row.empty(); }
/// current keys /// current keys
JoinKeyRow keys[2]; JoinKeyRow keys[2];
@ -118,8 +91,8 @@ public:
Chunk chunk; Chunk chunk;
}; };
AllJoinState(const SortCursorImpl & lcursor, size_t lpos, AllJoinState(const FullMergeJoinCursor & lcursor, size_t lpos,
const SortCursorImpl & rcursor, size_t rpos) const FullMergeJoinCursor & rcursor, size_t rpos)
: keys{JoinKeyRow(lcursor, lpos), JoinKeyRow(rcursor, rpos)} : keys{JoinKeyRow(lcursor, lpos), JoinKeyRow(rcursor, rpos)}
{ {
} }
@ -187,13 +160,32 @@ private:
size_t ridx = 0; size_t ridx = 0;
}; };
class AsofJoinState : boost::noncopyable
{
public:
void set(const FullMergeJoinCursor & rcursor, size_t rpos);
void reset();
bool hasMatch(const FullMergeJoinCursor & cursor, ASOFJoinInequality asof_inequality) const
{
if (value.empty())
return false;
return key.asofMatch(cursor, asof_inequality);
}
JoinKeyRow key;
Chunk value;
size_t value_row = 0;
};
/* /*
* Wrapper for SortCursorImpl * Wrapper for SortCursorImpl
*/ */
class FullMergeJoinCursor : boost::noncopyable class FullMergeJoinCursor : boost::noncopyable
{ {
public: public:
explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_); explicit FullMergeJoinCursor(const Block & sample_block_, const SortDescription & description_, bool is_asof = false);
bool fullyCompleted() const; bool fullyCompleted() const;
void setChunk(Chunk && chunk); void setChunk(Chunk && chunk);
@ -203,17 +195,31 @@ public:
SortCursorImpl * operator-> () { return &cursor; } SortCursorImpl * operator-> () { return &cursor; }
const SortCursorImpl * operator-> () const { return &cursor; } const SortCursorImpl * operator-> () const { return &cursor; }
SortCursorImpl & operator* () { return cursor; }
const SortCursorImpl & operator* () const { return cursor; }
SortCursorImpl cursor; SortCursorImpl cursor;
const Block & sampleBlock() const { return sample_block; } const Block & sampleBlock() const { return sample_block; }
Columns sampleColumns() const { return sample_block.getColumns(); } Columns sampleColumns() const { return sample_block.getColumns(); }
const IColumn * getAsofColumn() const
{
if (!asof_column_position)
return nullptr;
return cursor.all_columns[*asof_column_position];
}
String dump() const;
private: private:
Block sample_block; Block sample_block;
SortDescription desc; SortDescription desc;
Chunk current_chunk; Chunk current_chunk;
bool recieved_all_blocks = false; bool recieved_all_blocks = false;
std::optional<size_t> asof_column_position;
}; };
/* /*
@ -223,22 +229,33 @@ private:
class MergeJoinAlgorithm final : public IMergingAlgorithm class MergeJoinAlgorithm final : public IMergingAlgorithm
{ {
public: public:
explicit MergeJoinAlgorithm(JoinPtr table_join, const Blocks & input_headers, size_t max_block_size_); MergeJoinAlgorithm(JoinKind kind_,
JoinStrictness strictness_,
const TableJoin::JoinOnClause & on_clause_,
const Blocks & input_headers,
size_t max_block_size_);
MergeJoinAlgorithm(JoinPtr join_ptr, const Blocks & input_headers, size_t max_block_size_);
const char * getName() const override { return "MergeJoinAlgorithm"; } const char * getName() const override { return "MergeJoinAlgorithm"; }
void initialize(Inputs inputs) override; void initialize(Inputs inputs) override;
void consume(Input & input, size_t source_num) override; void consume(Input & input, size_t source_num) override;
Status merge() override; Status merge() override;
void logElapsed(double seconds); void setAsofInequality(ASOFJoinInequality asof_inequality_);
void logElapsed(double seconds);
private: private:
std::optional<Status> handleAnyJoinState(); std::optional<Status> handleAnyJoinState();
Status anyJoin(JoinKind kind); Status anyJoin();
std::optional<Status> handleAllJoinState(); std::optional<Status> handleAllJoinState();
Status allJoin(JoinKind kind); Status allJoin();
std::optional<Status> handleAsofJoinState();
Status asofJoin();
MutableColumns getEmptyResultColumns() const;
Chunk createBlockWithDefaults(size_t source_num); Chunk createBlockWithDefaults(size_t source_num);
Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const; Chunk createBlockWithDefaults(size_t source_num, size_t start, size_t num_rows) const;
@ -246,12 +263,15 @@ private:
std::unordered_map<size_t, size_t> left_to_right_key_remap; std::unordered_map<size_t, size_t> left_to_right_key_remap;
std::array<FullMergeJoinCursorPtr, 2> cursors; std::array<FullMergeJoinCursorPtr, 2> cursors;
ASOFJoinInequality asof_inequality = ASOFJoinInequality::None;
/// Keep some state to make connection between data in different blocks /// Keep some state to make handle data from different blocks
AnyJoinState any_join_state; AnyJoinState any_join_state;
std::unique_ptr<AllJoinState> all_join_state; std::unique_ptr<AllJoinState> all_join_state;
AsofJoinState asof_join_state;
JoinPtr table_join; JoinKind kind;
JoinStrictness strictness;
size_t max_block_size; size_t max_block_size;
int null_direction_hint = 1; int null_direction_hint = 1;
@ -281,12 +301,21 @@ public:
size_t max_block_size, size_t max_block_size,
UInt64 limit_hint = 0); UInt64 limit_hint = 0);
MergeJoinTransform(
JoinKind kind_,
JoinStrictness strictness_,
const TableJoin::JoinOnClause & on_clause_,
const Blocks & input_headers,
const Block & output_header,
size_t max_block_size,
UInt64 limit_hint_ = 0);
String getName() const override { return "MergeJoinTransform"; } String getName() const override { return "MergeJoinTransform"; }
void setAsofInequality(ASOFJoinInequality asof_inequality_) { algorithm.setAsofInequality(asof_inequality_); }
protected: protected:
void onFinish() override; void onFinish() override;
LoggerPtr log;
}; };
} }

View File

@ -0,0 +1,768 @@
#include <gtest/gtest.h>
#include <pcg_random.hpp>
#include <random>
#include <Poco/ConsoleChannel.h>
#include <Poco/Logger.h>
#include <Poco/AutoPtr.h>
#include <Columns/ColumnsNumber.h>
#include <Common/getRandomASCIIString.h>
#include <Common/randomSeed.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/TableJoin.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/Sinks/NullSink.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/Transforms/MergeJoinTransform.h>
#include <Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <QueryPipeline/QueryPipeline.h>
using namespace DB;
namespace
{
QueryPipeline buildJoinPipeline(
std::shared_ptr<ISource> left_source,
std::shared_ptr<ISource> right_source,
size_t key_length = 1,
JoinKind kind = JoinKind::Inner,
JoinStrictness strictness = JoinStrictness::All,
ASOFJoinInequality asof_inequality = ASOFJoinInequality::None)
{
Blocks inputs;
inputs.emplace_back(left_source->getPort().getHeader());
inputs.emplace_back(right_source->getPort().getHeader());
Block out_header;
for (const auto & input : inputs)
{
for (ColumnWithTypeAndName column : input)
{
if (&input == &inputs.front())
column.name = "t1." + column.name;
else
column.name = "t2." + column.name;
out_header.insert(column);
}
}
TableJoin::JoinOnClause on_clause;
for (size_t i = 0; i < key_length; ++i)
{
on_clause.key_names_left.emplace_back(inputs[0].getByPosition(i).name);
on_clause.key_names_right.emplace_back(inputs[1].getByPosition(i).name);
}
auto joining = std::make_shared<MergeJoinTransform>(
kind,
strictness,
on_clause,
inputs, out_header, /* max_block_size = */ 0);
if (asof_inequality != ASOFJoinInequality::None)
joining->setAsofInequality(asof_inequality);
chassert(joining->getInputs().size() == 2);
connect(left_source->getPort(), joining->getInputs().front());
connect(right_source->getPort(), joining->getInputs().back());
auto * output_port = &joining->getOutputPort();
auto processors = std::make_shared<Processors>();
processors->emplace_back(std::move(left_source));
processors->emplace_back(std::move(right_source));
processors->emplace_back(std::move(joining));
QueryPipeline pipeline(QueryPlanResourceHolder{}, processors, output_port);
return pipeline;
}
std::shared_ptr<ISource> oneColumnSource(const std::vector<std::vector<UInt64>> & values)
{
Block header = {
ColumnWithTypeAndName(std::make_shared<DataTypeUInt64>(), "key"),
ColumnWithTypeAndName(std::make_shared<DataTypeUInt64>(), "idx"),
};
UInt64 idx = 0;
Chunks chunks;
for (const auto & chunk_values : values)
{
auto key_column = ColumnUInt64::create();
auto idx_column = ColumnUInt64::create();
for (auto n : chunk_values)
{
key_column->insertValue(n);
idx_column->insertValue(idx);
++idx;
}
chunks.emplace_back(Chunk(Columns{std::move(key_column), std::move(idx_column)}, chunk_values.size()));
}
return std::make_shared<SourceFromChunks>(header, std::move(chunks));
}
class SourceChunksBuilder
{
public:
explicit SourceChunksBuilder(const Block & header_)
: header(header_)
{
current_chunk = header.cloneEmptyColumns();
chassert(!current_chunk.empty());
}
void setBreakProbability(pcg64 & rng_)
{
/// random probability with possibility to have exact 0.0 and 1.0 values
break_prob = std::uniform_int_distribution<size_t>(0, 5)(rng_) / static_cast<double>(5);
rng = &rng_;
}
void addRow(const std::vector<Field> & row)
{
chassert(row.size() == current_chunk.size());
for (size_t i = 0; i < current_chunk.size(); ++i)
current_chunk[i]->insert(row[i]);
if (rng && std::uniform_real_distribution<>(0.0, 1.0)(*rng) < break_prob)
addChunk();
}
void addChunk()
{
if (current_chunk.front()->empty())
return;
size_t rows = current_chunk.front()->size();
chunks.emplace_back(std::move(current_chunk), rows);
current_chunk = header.cloneEmptyColumns();
}
std::shared_ptr<ISource> getSource()
{
addChunk();
/// copy chunk to allow reusing same builder
Chunks chunks_copy;
chunks_copy.reserve(chunks.size());
for (const auto & chunk : chunks)
chunks_copy.emplace_back(chunk.clone());
return std::make_shared<SourceFromChunks>(header, std::move(chunks_copy));
}
private:
Block header;
Chunks chunks;
MutableColumns current_chunk;
pcg64 * rng = nullptr;
double break_prob = 0.0;
};
std::vector<std::vector<Field>> getValuesFromBlock(const Block & block, const Names & names)
{
std::vector<std::vector<Field>> result;
for (size_t i = 0; i < block.rows(); ++i)
{
auto & row = result.emplace_back();
for (const auto & name : names)
block.getByName(name).column->get(i, row.emplace_back());
}
return result;
}
Block executePipeline(QueryPipeline && pipeline)
{
PullingPipelineExecutor executor(pipeline);
Blocks result_blocks;
while (true)
{
Block block;
bool is_ok = executor.pull(block);
if (!is_ok)
break;
result_blocks.emplace_back(std::move(block));
}
return concatenateBlocks(result_blocks);
}
template <typename T>
void assertColumnVectorEq(const typename ColumnVector<T>::Container & expected, const Block & block, const std::string & name)
{
const auto * actual = typeid_cast<const ColumnVector<T> *>(block.getByName(name).column.get());
ASSERT_TRUE(actual) << "unexpected column type: " << block.getByName(name).column->dumpStructure() << "expected: " << typeid(ColumnVector<T>).name();
auto get_first_diff = [&]() -> String
{
const auto & actual_data = actual->getData();
size_t num_rows = std::min(expected.size(), actual_data.size());
for (size_t i = 0; i < num_rows; ++i)
{
if (expected[i] != actual_data[i])
return fmt::format(", expected: {}, actual: {} at row {}", expected[i], actual_data[i], i);
}
return "";
};
EXPECT_EQ(actual->getData().size(), expected.size());
ASSERT_EQ(actual->getData(), expected) << "column name: " << name << get_first_diff();
}
template <typename T>
void assertColumnEq(const IColumn & expected, const Block & block, const std::string & name)
{
const ColumnPtr & actual = block.getByName(name).column;
ASSERT_TRUE(checkColumn<T>(*actual));
ASSERT_TRUE(checkColumn<T>(expected));
EXPECT_EQ(actual->size(), expected.size());
auto dump_val = [](const IColumn & col, size_t i) -> String
{
Field value;
col.get(i, value);
return value.dump();
};
size_t num_rows = std::min(actual->size(), expected.size());
for (size_t i = 0; i < num_rows; ++i)
ASSERT_EQ(actual->compareAt(i, i, expected, 1), 0) << dump_val(*actual, i) << " != " << dump_val(expected, i) << " at row " << i;
}
template <typename T>
T getRandomFrom(pcg64 & rng, const std::initializer_list<T> & opts)
{
std::vector<T> options(opts.begin(), opts.end());
size_t idx = std::uniform_int_distribution<size_t>(0, options.size() - 1)(rng);
return options[idx];
}
void generateNextKey(pcg64 & rng, UInt64 & k1, String & k2)
{
size_t str_len = std::uniform_int_distribution<>(1, 10)(rng);
String new_k2 = getRandomASCIIString(str_len, rng);
if (new_k2.compare(k2) <= 0)
++k1;
k2 = new_k2;
}
bool isStrict(ASOFJoinInequality inequality)
{
return inequality == ASOFJoinInequality::Less || inequality == ASOFJoinInequality::Greater;
}
}
class FullSortingJoinTest : public ::testing::Test
{
public:
FullSortingJoinTest() = default;
void SetUp() override
{
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
Poco::Logger::root().setChannel(channel);
if (const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe)
Poco::Logger::root().setLevel(test_log_level);
else
Poco::Logger::root().setLevel("none");
UInt64 seed = randomSeed();
if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe)
seed = std::stoull(random_seed);
std::cout << "TEST_RANDOM_SEED=" << seed << std::endl;
rng = pcg64(seed);
}
void TearDown() override
{
}
pcg64 rng;
};
TEST_F(FullSortingJoinTest, AllAnyOneKey)
try
{
{
SCOPED_TRACE("Inner All");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {1, 2, 3, 4, 5} }),
oneColumnSource({ {1}, {2}, {3}, {4}, {5} }),
1, JoinKind::Inner, JoinStrictness::All));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t2.idx");
}
{
SCOPED_TRACE("Inner Any");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {1, 2, 3, 4, 5} }),
oneColumnSource({ {1}, {2}, {3}, {4}, {5} }),
1, JoinKind::Inner, JoinStrictness::Any));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4}), result, "t2.idx");
}
{
SCOPED_TRACE("Inner All");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }),
oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }),
1, JoinKind::Inner, JoinStrictness::All));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 0, 1, 2, 3, 3, 4, 5}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({3, 3, 3, 4, 4, 4, 3, 4, 5, 5}), result, "t2.idx");
}
{
SCOPED_TRACE("Inner Any");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }),
oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }),
1, JoinKind::Inner, JoinStrictness::Any));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 4}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({3, 5}), result, "t2.idx");
}
{
SCOPED_TRACE("Inner Any");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {2, 2, 2, 2}, {3}, {3, 5} }),
oneColumnSource({ {1, 1, 1, 2}, {2}, {3, 4} }),
1, JoinKind::Inner, JoinStrictness::Any));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 4}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({3, 5}), result, "t2.idx");
}
{
SCOPED_TRACE("Left Any");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {2, 2, 2}, {2, 3}, {3, 5} }),
oneColumnSource({ {1, 1, 1}, {2, 2}, {3, 4} }),
1, JoinKind::Left, JoinStrictness::Any));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4, 5, 6}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({3, 3, 3, 3, 5, 5, 0}), result, "t2.idx");
}
{
SCOPED_TRACE("Left Any");
Block result = executePipeline(buildJoinPipeline(
oneColumnSource({ {2, 2, 2, 2}, {3}, {3, 5} }),
oneColumnSource({ {1, 1, 1, 2}, {2}, {3, 4} }),
1, JoinKind::Left, JoinStrictness::Any));
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({0, 1, 2, 3, 4, 5, 6}), result, "t1.idx");
assertColumnVectorEq<UInt64>(ColumnUInt64::Container({3, 3, 3, 3, 5, 5, 0}), result, "t2.idx");
}
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}
TEST_F(FullSortingJoinTest, AnySimple)
try
{
JoinKind kind = getRandomFrom(rng, {JoinKind::Inner, JoinKind::Left, JoinKind::Right});
SourceChunksBuilder left_source({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeString>(), "attr"},
});
SourceChunksBuilder right_source({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeString>(), "attr"},
});
left_source.setBreakProbability(rng);
right_source.setBreakProbability(rng);
size_t num_keys = std::uniform_int_distribution<>(100, 1000)(rng);
auto expected_left = ColumnString::create();
auto expected_right = ColumnString::create();
UInt64 k1 = 1;
String k2;
auto get_attr = [&](const String & side, size_t idx) -> String
{
return toString(k1) + "_" + k2 + "_" + side + "_" + toString(idx);
};
for (size_t i = 0; i < num_keys; ++i)
{
generateNextKey(rng, k1, k2);
/// Key is present in left, right or both tables. Both tables is more probable.
size_t key_presence = std::uniform_int_distribution<>(0, 10)(rng);
size_t num_rows_left = key_presence == 0 ? 0 : std::uniform_int_distribution<>(1, 10)(rng);
for (size_t j = 0; j < num_rows_left; ++j)
left_source.addRow({k1, k2, get_attr("left", j)});
size_t num_rows_right = key_presence == 1 ? 0 : std::uniform_int_distribution<>(1, 10)(rng);
for (size_t j = 0; j < num_rows_right; ++j)
right_source.addRow({k1, k2, get_attr("right", j)});
String left_attr = num_rows_left ? get_attr("left", 0) : "";
String right_attr = num_rows_right ? get_attr("right", 0) : "";
if (kind == JoinKind::Inner && num_rows_left && num_rows_right)
{
expected_left->insert(left_attr);
expected_right->insert(right_attr);
}
else if (kind == JoinKind::Left)
{
for (size_t j = 0; j < num_rows_left; ++j)
{
expected_left->insert(get_attr("left", j));
expected_right->insert(right_attr);
}
}
else if (kind == JoinKind::Right)
{
for (size_t j = 0; j < num_rows_right; ++j)
{
expected_left->insert(left_attr);
expected_right->insert(get_attr("right", j));
}
}
}
Block result_block = executePipeline(buildJoinPipeline(
left_source.getSource(), right_source.getSource(), /* key_length = */ 2,
kind, JoinStrictness::Any));
assertColumnEq<ColumnString>(*expected_left, result_block, "t1.attr");
assertColumnEq<ColumnString>(*expected_right, result_block, "t2.attr");
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}
TEST_F(FullSortingJoinTest, AsofSimple)
try
{
SourceChunksBuilder left_source({
{std::make_shared<DataTypeString>(), "key"},
{std::make_shared<DataTypeUInt64>(), "t"},
});
left_source.addRow({"AMZN", 3});
left_source.addRow({"AMZN", 4});
left_source.addRow({"AMZN", 6});
left_source.addRow({"SBUX", 10});
SourceChunksBuilder right_source({
{std::make_shared<DataTypeString>(), "key"},
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeUInt64>(), "value"},
});
right_source.addRow({"AAPL", 1, 97});
right_source.addChunk();
right_source.addRow({"AAPL", 2, 98});
right_source.addRow({"AAPL", 3, 99});
right_source.addRow({"AMZN", 1, 100});
right_source.addRow({"AMZN", 2, 110});
right_source.addChunk();
right_source.addRow({"AMZN", 2, 110});
right_source.addChunk();
right_source.addRow({"AMZN", 4, 130});
right_source.addRow({"AMZN", 5, 140});
right_source.addRow({"SBUX", 8, 180});
right_source.addChunk();
right_source.addRow({"SBUX", 9, 190});
{
Block result_block = executePipeline(buildJoinPipeline(
left_source.getSource(), right_source.getSource(), /* key_length = */ 2,
JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals));
auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"});
ASSERT_EQ(values, (std::vector<std::vector<Field>>{
{"AMZN", 3u, 4u, 130u},
{"AMZN", 4u, 4u, 130u},
}));
}
{
Block result_block = executePipeline(buildJoinPipeline(
left_source.getSource(), right_source.getSource(), /* key_length = */ 2,
JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::GreaterOrEquals));
auto values = getValuesFromBlock(result_block, {"t1.key", "t1.t", "t2.t", "t2.value"});
ASSERT_EQ(values, (std::vector<std::vector<Field>>{
{"AMZN", 3u, 2u, 110u},
{"AMZN", 4u, 4u, 130u},
{"AMZN", 6u, 5u, 140u},
{"SBUX", 10u, 9u, 190u},
}));
}
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}
TEST_F(FullSortingJoinTest, AsofOnlyColumn)
try
{
auto left_source = oneColumnSource({ {3}, {3, 3, 3}, {3, 5, 5, 6}, {9, 9}, {10, 20} });
SourceChunksBuilder right_source_builder({
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeUInt64>(), "value"},
});
right_source_builder.setBreakProbability(rng);
for (const auto & row : std::vector<std::vector<Field>>{ {1, 101}, {2, 102}, {4, 104}, {5, 105}, {11, 111}, {15, 115} })
right_source_builder.addRow(row);
auto right_source = right_source_builder.getSource();
auto pipeline = buildJoinPipeline(
left_source, right_source, /* key_length = */ 1,
JoinKind::Inner, JoinStrictness::Asof, ASOFJoinInequality::LessOrEquals);
Block result_block = executePipeline(std::move(pipeline));
ASSERT_EQ(
assert_cast<const ColumnUInt64 *>(result_block.getByName("t1.key").column.get())->getData(),
(ColumnUInt64::Container{3, 3, 3, 3, 3, 5, 5, 6, 9, 9, 10})
);
ASSERT_EQ(
assert_cast<const ColumnUInt64 *>(result_block.getByName("t2.t").column.get())->getData(),
(ColumnUInt64::Container{4, 4, 4, 4, 4, 5, 5, 11, 11, 11, 11})
);
ASSERT_EQ(
assert_cast<const ColumnUInt64 *>(result_block.getByName("t2.value").column.get())->getData(),
(ColumnUInt64::Container{104, 104, 104, 104, 104, 105, 105, 111, 111, 111, 111})
);
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}
TEST_F(FullSortingJoinTest, AsofLessGeneratedTestData)
try
{
/// Generate data random and build expected result at the same time.
/// Test specific combinations of join kind and inequality per each run
auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left });
auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Less, ASOFJoinInequality::LessOrEquals });
SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality));
/// Key is complex, `k1, k2` for equality and `t` for asof
SourceChunksBuilder left_source_builder({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeInt64>(), "attr"},
});
SourceChunksBuilder right_source_builder({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeInt64>(), "attr"},
});
/// How small generated block should be
left_source_builder.setBreakProbability(rng);
right_source_builder.setBreakProbability(rng);
/// We are going to generate sorted data and remember expected result
ColumnInt64::Container expected;
UInt64 k1 = 1;
String k2;
auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng);
for (size_t key_num = 0; key_num < key_num_total; ++key_num)
{
/// Generate new key greater than previous
generateNextKey(rng, k1, k2);
Int64 left_t = 0;
/// Generate several rows for the key
size_t num_left_rows = std::uniform_int_distribution<>(1, 100)(rng);
for (size_t i = 0; i < num_left_rows; ++i)
{
/// t is strictly greater than previous
left_t += std::uniform_int_distribution<>(1, 10)(rng);
auto left_arrtibute_value = 10 * left_t;
left_source_builder.addRow({k1, k2, left_t, left_arrtibute_value});
expected.push_back(left_arrtibute_value);
auto num_matches = 1 + std::poisson_distribution<>(4)(rng);
/// Generate several matches in the right table
auto right_t = left_t;
for (size_t j = 0; j < num_matches; ++j)
{
int min_step = isStrict(asof_inequality) ? 1 : 0;
right_t += std::uniform_int_distribution<>(min_step, 3)(rng);
/// First row should match
bool is_match = j == 0;
right_source_builder.addRow({k1, k2, right_t, is_match ? 10 * left_arrtibute_value : -1});
}
/// Next left_t should be greater than right_t not to match with previous rows
left_t = right_t;
}
/// generate some rows with greater left_t to check that they are not matched
num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 100)(rng) : 0;
for (size_t i = 0; i < num_left_rows; ++i)
{
left_t += std::uniform_int_distribution<>(1, 10)(rng);
left_source_builder.addRow({k1, k2, left_t, -10 * left_t});
if (join_kind == JoinKind::Left)
expected.push_back(-10 * left_t);
}
}
Block result_block = executePipeline(buildJoinPipeline(
left_source_builder.getSource(), right_source_builder.getSource(),
/* key_length = */ 3,
join_kind, JoinStrictness::Asof, asof_inequality));
assertColumnVectorEq<Int64>(expected, result_block, "t1.attr");
for (auto & e : expected)
/// Non matched rows from left table have negative attr
/// Value if attribute in right table is 10 times greater than in left table
e = e < 0 ? 0 : 10 * e;
assertColumnVectorEq<Int64>(expected, result_block, "t2.attr");
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}
TEST_F(FullSortingJoinTest, AsofGreaterGeneratedTestData)
try
{
/// Generate data random and build expected result at the same time.
/// Test specific combinations of join kind and inequality per each run
auto join_kind = getRandomFrom(rng, { JoinKind::Inner, JoinKind::Left });
auto asof_inequality = getRandomFrom(rng, { ASOFJoinInequality::Greater, ASOFJoinInequality::GreaterOrEquals });
SCOPED_TRACE(fmt::format("{} {}", join_kind, asof_inequality));
SourceChunksBuilder left_source_builder({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeInt64>(), "attr"},
});
SourceChunksBuilder right_source_builder({
{std::make_shared<DataTypeUInt64>(), "k1"},
{std::make_shared<DataTypeString>(), "k2"},
{std::make_shared<DataTypeUInt64>(), "t"},
{std::make_shared<DataTypeInt64>(), "attr"},
});
left_source_builder.setBreakProbability(rng);
right_source_builder.setBreakProbability(rng);
ColumnInt64::Container expected;
UInt64 k1 = 1;
String k2;
UInt64 left_t = 0;
auto key_num_total = std::uniform_int_distribution<>(1, 1000)(rng);
for (size_t key_num = 0; key_num < key_num_total; ++key_num)
{
/// Generate new key greater than previous
generateNextKey(rng, k1, k2);
/// Generate some rows with smaller left_t to check that they are not matched
size_t num_left_rows = std::bernoulli_distribution(0.5)(rng) ? std::uniform_int_distribution<>(1, 100)(rng) : 0;
for (size_t i = 0; i < num_left_rows; ++i)
{
left_t += std::uniform_int_distribution<>(1, 10)(rng);
left_source_builder.addRow({k1, k2, left_t, -10 * left_t});
if (join_kind == JoinKind::Left)
expected.push_back(-10 * left_t);
}
if (std::bernoulli_distribution(0.1)(rng))
continue;
size_t num_right_matches = std::uniform_int_distribution<>(1, 10)(rng);
auto right_t = left_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 0 : 1, 10)(rng);
auto attribute_value = 10 * right_t;
for (size_t j = 0; j < num_right_matches; ++j)
{
right_t += std::uniform_int_distribution<>(0, 3)(rng);
bool is_match = j == num_right_matches - 1;
right_source_builder.addRow({k1, k2, right_t, is_match ? 10 * attribute_value : -1});
}
/// Next left_t should be greater than (or equals) right_t to match with previous rows
left_t = right_t + std::uniform_int_distribution<>(isStrict(asof_inequality) ? 1 : 0, 100)(rng);
size_t num_left_matches = std::uniform_int_distribution<>(1, 100)(rng);
for (size_t j = 0; j < num_left_matches; ++j)
{
left_t += std::uniform_int_distribution<>(0, 3)(rng);
left_source_builder.addRow({k1, k2, left_t, attribute_value});
expected.push_back(attribute_value);
}
}
Block result_block = executePipeline(buildJoinPipeline(
left_source_builder.getSource(), right_source_builder.getSource(),
/* key_length = */ 3,
join_kind, JoinStrictness::Asof, asof_inequality));
assertColumnVectorEq<Int64>(expected, result_block, "t1.attr");
for (auto & e : expected)
/// Non matched rows from left table have negative attr
/// Value if attribute in right table is 10 times greater than in left table
e = e < 0 ? 0 : 10 * e;
assertColumnVectorEq<Int64>(expected, result_block, "t2.attr");
}
catch (Exception & e)
{
std::cout << e.getStackTraceString() << std::endl;
throw;
}

View File

@ -43,7 +43,6 @@ class R2MountPoint:
self.bucket_name = self._PROD_BUCKET_NAME self.bucket_name = self._PROD_BUCKET_NAME
self.aux_mount_options = "" self.aux_mount_options = ""
self.async_mount = False
if self.app == MountPointApp.S3FS: if self.app == MountPointApp.S3FS:
self.cache_dir = "/home/ubuntu/s3fs_cache" self.cache_dir = "/home/ubuntu/s3fs_cache"
# self.aux_mount_options += "-o nomodtime " if self.NOMODTIME else "" not for s3fs # self.aux_mount_options += "-o nomodtime " if self.NOMODTIME else "" not for s3fs
@ -57,7 +56,6 @@ class R2MountPoint:
self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}"
elif self.app == MountPointApp.RCLONE: elif self.app == MountPointApp.RCLONE:
# run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return # run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return
self.async_mount = True
self.cache_dir = "/home/ubuntu/rclone_cache" self.cache_dir = "/home/ubuntu/rclone_cache"
self.aux_mount_options += "--no-modtime " if self.NOMODTIME else "" self.aux_mount_options += "--no-modtime " if self.NOMODTIME else ""
self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose
@ -85,10 +83,12 @@ class R2MountPoint:
Shell.run(_UNMOUNT_CMD) Shell.run(_UNMOUNT_CMD)
Shell.run(_MKDIR_CMD) Shell.run(_MKDIR_CMD)
Shell.run(_MKDIR_FOR_CACHE) Shell.run(_MKDIR_FOR_CACHE)
# didn't manage to use simple run() and not block or fail if self.app == MountPointApp.S3FS:
Shell.run_as_daemon(self.mount_cmd) Shell.run(self.mount_cmd, check=True)
if self.async_mount: else:
time.sleep(3) # didn't manage to use simple run() and without blocking or failure
Shell.run_as_daemon(self.mount_cmd)
time.sleep(3)
Shell.run(_TEST_MOUNT_CMD, check=True) Shell.run(_TEST_MOUNT_CMD, check=True)
@classmethod @classmethod
@ -107,6 +107,7 @@ class DebianArtifactory:
_PROD_REPO_URL = "https://packages.clickhouse.com/deb" _PROD_REPO_URL = "https://packages.clickhouse.com/deb"
def __init__(self, release_info: ReleaseInfo, dry_run: bool): def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.release_info = release_info
self.codename = release_info.codename self.codename = release_info.codename
self.version = release_info.version self.version = release_info.version
if dry_run: if dry_run:
@ -154,9 +155,8 @@ class DebianArtifactory:
print("Running test command:") print("Running test command:")
print(f" {cmd}") print(f" {cmd}")
Shell.run(cmd, check=True) Shell.run(cmd, check=True)
release_info = ReleaseInfo.from_file() self.release_info.debian_command = debian_command
release_info.debian_command = debian_command self.release_info.dump()
release_info.dump()
def _copy_if_not_exists(src: Path, dst: Path) -> Path: def _copy_if_not_exists(src: Path, dst: Path) -> Path:
@ -177,6 +177,7 @@ class RpmArtifactory:
_SIGN_KEY = "885E2BDCF96B0B45ABF058453E4AD4719DDE9A38" _SIGN_KEY = "885E2BDCF96B0B45ABF058453E4AD4719DDE9A38"
def __init__(self, release_info: ReleaseInfo, dry_run: bool): def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.release_info = release_info
self.codename = release_info.codename self.codename = release_info.codename
self.version = release_info.version self.version = release_info.version
if dry_run: if dry_run:
@ -230,9 +231,8 @@ class RpmArtifactory:
print("Running test command:") print("Running test command:")
print(f" {cmd}") print(f" {cmd}")
Shell.run(cmd, check=True) Shell.run(cmd, check=True)
release_info = ReleaseInfo.from_file() self.release_info.rpm_command = rpm_command
release_info.rpm_command = rpm_command self.release_info.dump()
release_info.dump()
class TgzArtifactory: class TgzArtifactory:
@ -240,6 +240,7 @@ class TgzArtifactory:
_PROD_REPO_URL = "https://packages.clickhouse.com/tgz" _PROD_REPO_URL = "https://packages.clickhouse.com/tgz"
def __init__(self, release_info: ReleaseInfo, dry_run: bool): def __init__(self, release_info: ReleaseInfo, dry_run: bool):
self.release_info = release_info
self.codename = release_info.codename self.codename = release_info.codename
self.version = release_info.version self.version = release_info.version
if dry_run: if dry_run:
@ -290,9 +291,8 @@ class TgzArtifactory:
expected_checksum == actual_checksum expected_checksum == actual_checksum
), f"[{actual_checksum} != {expected_checksum}]" ), f"[{actual_checksum} != {expected_checksum}]"
Shell.run("rm /tmp/tmp.tgz*") Shell.run("rm /tmp/tmp.tgz*")
release_info = ReleaseInfo.from_file() self.release_info.tgz_command = cmd
release_info.tgz_command = cmd self.release_info.dump()
release_info.dump()
def parse_args() -> argparse.Namespace: def parse_args() -> argparse.Namespace:
@ -340,9 +340,7 @@ def parse_args() -> argparse.Namespace:
if __name__ == "__main__": if __name__ == "__main__":
args = parse_args() args = parse_args()
assert args.dry_run
release_info = ReleaseInfo.from_file()
""" """
Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve:
ERROR : IO error: NotImplemented: versionId not implemented ERROR : IO error: NotImplemented: versionId not implemented
@ -350,26 +348,38 @@ if __name__ == "__main__":
""" """
mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run)
if args.export_debian: if args.export_debian:
with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_DEB) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.EXPORT_DEB
) as release_info:
mp.init() mp.init()
DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() DebianArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown() mp.teardown()
if args.export_rpm: if args.export_rpm:
with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_RPM) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.EXPORT_RPM
) as release_info:
mp.init() mp.init()
RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() RpmArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown() mp.teardown()
if args.export_tgz: if args.export_tgz:
with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_TGZ) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.EXPORT_TGZ
) as release_info:
mp.init() mp.init()
TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() TgzArtifactory(release_info, dry_run=args.dry_run).export_packages()
mp.teardown() mp.teardown()
if args.test_debian: if args.test_debian:
with ReleaseContextManager(release_progress=ReleaseProgress.TEST_DEB) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.TEST_DEB
) as release_info:
DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() DebianArtifactory(release_info, dry_run=args.dry_run).test_packages()
if args.test_tgz: if args.test_tgz:
with ReleaseContextManager(release_progress=ReleaseProgress.TEST_TGZ) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.TEST_TGZ
) as release_info:
TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() TgzArtifactory(release_info, dry_run=args.dry_run).test_packages()
if args.test_rpm: if args.test_rpm:
with ReleaseContextManager(release_progress=ReleaseProgress.TEST_RPM) as _: with ReleaseContextManager(
release_progress=ReleaseProgress.TEST_RPM
) as release_info:
RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() RpmArtifactory(release_info, dry_run=args.dry_run).test_packages()

View File

@ -191,7 +191,7 @@ def main():
title=f"Auto Release Status for {release_info.release_branch}", title=f"Auto Release Status for {release_info.release_branch}",
body=release_info.to_dict(), body=release_info.to_dict(),
) )
if args.post_auto_release_complete: elif args.post_auto_release_complete:
assert args.wf_status, "--wf-status Required with --post-auto-release-complete" assert args.wf_status, "--wf-status Required with --post-auto-release-complete"
if args.wf_status != SUCCESS: if args.wf_status != SUCCESS:
CIBuddy(dry_run=False).post_job_error( CIBuddy(dry_run=False).post_job_error(

View File

@ -182,10 +182,11 @@ class Shell:
check=False, check=False,
) )
if result.returncode == 0: if result.returncode == 0:
print(f"stdout: {result.stdout.strip()}")
res = result.stdout res = result.stdout
else: else:
print( print(
f"ERROR: stdout {result.stdout.strip()}, stderr {result.stderr.strip()}" f"ERROR: stdout: {result.stdout.strip()}, stderr: {result.stderr.strip()}"
) )
if check: if check:
assert result.returncode == 0 assert result.returncode == 0

View File

@ -43,6 +43,7 @@ class ReleaseProgress:
TEST_TGZ = "test TGZ packages" TEST_TGZ = "test TGZ packages"
TEST_RPM = "test RPM packages" TEST_RPM = "test RPM packages"
TEST_DEB = "test DEB packages" TEST_DEB = "test DEB packages"
COMPLETED = "completed"
class ReleaseProgressDescription: class ReleaseProgressDescription:
@ -108,6 +109,12 @@ class ReleaseInfo:
release_progress: str = "" release_progress: str = ""
progress_description: str = "" progress_description: str = ""
def is_patch(self):
return self.release_branch != "master"
def is_new_release_branch(self):
return self.release_branch == "master"
@staticmethod @staticmethod
def from_file() -> "ReleaseInfo": def from_file() -> "ReleaseInfo":
with open(RELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: with open(RELEASE_INFO_FILE, "r", encoding="utf-8") as json_file:
@ -126,12 +133,12 @@ class ReleaseInfo:
release_tag = None release_tag = None
previous_release_tag = None previous_release_tag = None
previous_release_sha = None previous_release_sha = None
codename = None codename = ""
assert release_type in ("patch", "new") assert release_type in ("patch", "new")
if release_type == "new": if release_type == "new":
# check commit_ref is right and on a right branch # check commit_ref is right and on a right branch
Shell.run( Shell.run(
f"git merge-base --is-ancestor origin/{commit_ref} origin/master", f"git merge-base --is-ancestor {commit_ref} origin/master",
check=True, check=True,
) )
with checkout(commit_ref): with checkout(commit_ref):
@ -146,9 +153,6 @@ class ReleaseInfo:
git.latest_tag == expected_prev_tag git.latest_tag == expected_prev_tag
), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]" ), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]"
release_tag = version.describe release_tag = version.describe
codename = (
VersionType.STABLE
) # dummy value (artifactory won't be updated for new release)
previous_release_tag = expected_prev_tag previous_release_tag = expected_prev_tag
previous_release_sha = Shell.run_strict( previous_release_sha = Shell.run_strict(
f"git rev-parse {previous_release_tag}" f"git rev-parse {previous_release_tag}"
@ -205,7 +209,7 @@ class ReleaseInfo:
and commit_sha and commit_sha
and release_tag and release_tag
and version and version
and codename in ("lts", "stable") and (codename in ("lts", "stable") or release_type == "new")
) )
self.release_branch = release_branch self.release_branch = release_branch
@ -320,24 +324,27 @@ class ReleaseInfo:
Shell.run( Shell.run(
f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'"
) )
self.version_bump_pr = GHActions.get_pr_url_by_branch( self.version_bump_pr = "dry-run"
repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors else:
) self.version_bump_pr = GHActions.get_pr_url_by_branch(
repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors
)
def update_release_info(self, dry_run: bool) -> "ReleaseInfo": def update_release_info(self, dry_run: bool) -> "ReleaseInfo":
branch = f"auto/{release_info.release_tag}" if self.release_branch != "master":
if not dry_run: branch = f"auto/{release_info.release_tag}"
url = GHActions.get_pr_url_by_branch(repo=GITHUB_REPOSITORY, branch=branch) if not dry_run:
else: url = GHActions.get_pr_url_by_branch(
url = "dry-run" repo=GITHUB_REPOSITORY, branch=branch
)
print(f"ChangeLog PR url [{url}]") else:
self.changelog_pr = url url = "dry-run"
print(f"Release url [{url}]") print(f"ChangeLog PR url [{url}]")
self.release_url = ( self.changelog_pr = url
f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" print(f"Release url [{url}]")
) self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}"
self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.release_branch} clickhouse --version" if self.release_progress == ReleaseProgress.COMPLETED:
self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version"
self.dump() self.dump()
return self return self
@ -712,13 +719,22 @@ if __name__ == "__main__":
if args.post_status: if args.post_status:
release_info = ReleaseInfo.from_file() release_info = ReleaseInfo.from_file()
release_info.update_release_info(dry_run=args.dry_run) release_info.update_release_info(dry_run=args.dry_run)
if release_info.debian_command: if release_info.is_new_release_branch():
title = "New release branch"
else:
title = "New release"
if (
release_info.progress_description == ReleaseProgressDescription.OK
and release_info.release_progress == ReleaseProgress.COMPLETED
):
title = "Completed: " + title
CIBuddy(dry_run=args.dry_run).post_done( CIBuddy(dry_run=args.dry_run).post_done(
f"New release issued", dataclasses.asdict(release_info) title, dataclasses.asdict(release_info)
) )
else: else:
title = "Failed: " + title
CIBuddy(dry_run=args.dry_run).post_critical( CIBuddy(dry_run=args.dry_run).post_critical(
f"Failed to issue new release", dataclasses.asdict(release_info) title, dataclasses.asdict(release_info)
) )
if args.set_progress_started: if args.set_progress_started:

View File

@ -50,7 +50,7 @@ set -uo pipefail
# set accordingly to a runner role # # set accordingly to a runner role #
#################################### ####################################
echo "Running init v1" echo "Running init v1.1"
export DEBIAN_FRONTEND=noninteractive export DEBIAN_FRONTEND=noninteractive
export RUNNER_HOME=/home/ubuntu/actions-runner export RUNNER_HOME=/home/ubuntu/actions-runner
@ -66,6 +66,14 @@ bash /usr/local/share/scripts/init-network.sh
RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text) RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text)
LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE" LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE"
export LABELS export LABELS
echo "Instance Labels: $LABELS"
LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle)
export LIFE_CYCLE
echo "Instance lifecycle: $LIFE_CYCLE"
INSTANCE_TYPE=$(ec2metadata --instance-type)
echo "Instance type: $INSTANCE_TYPE"
# Refresh CloudWatch agent config # Refresh CloudWatch agent config
aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json
@ -124,10 +132,6 @@ terminate_decrease_and_exit() {
declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh
check_spot_instance_is_old() { check_spot_instance_is_old() {
# This function should be executed ONLY BETWEEN runnings.
# It's unsafe to execute while the runner is working!
local LIFE_CYCLE
LIFE_CYCLE=$(curl -s --fail http://169.254.169.254/latest/meta-data/instance-life-cycle)
if [ "$LIFE_CYCLE" == "spot" ]; then if [ "$LIFE_CYCLE" == "spot" ]; then
local UPTIME local UPTIME
UPTIME=$(< /proc/uptime) UPTIME=$(< /proc/uptime)

View File

@ -43,15 +43,10 @@ def started_cluster():
config = """<clickhouse> config = """<clickhouse>
<openSSL> <openSSL>
<client> <client>
<verificationMode>none</verificationMode> <verificationMode>strict</verificationMode>
<certificateFile>{certificateFile}</certificateFile> <certificateFile>{certificateFile}</certificateFile>
<privateKeyFile>{privateKeyFile}</privateKeyFile> <privateKeyFile>{privateKeyFile}</privateKeyFile>
<caConfig>{caConfig}</caConfig> <caConfig>{caConfig}</caConfig>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client> </client>
</openSSL> </openSSL>
</clickhouse>""" </clickhouse>"""

View File

@ -4,9 +4,7 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance( node = cluster.add_instance("node", main_configs=["config/config.xml"])
"node1", main_configs=["config/config.xml"], with_zookeeper=True
)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
@ -21,23 +19,28 @@ def started_cluster():
def test_table_db_limit(started_cluster): def test_table_db_limit(started_cluster):
for i in range(10): # By the way, default database already exists.
node1.query("create database db{}".format(i)) for i in range(9):
node.query("create database db{}".format(i))
with pytest.raises(QueryRuntimeException) as exp_info: with pytest.raises(QueryRuntimeException) as exp_info:
node1.query("create database db_exp".format(i)) node.query("create database db_exp".format(i))
assert "TOO_MANY_DATABASES" in str(exp_info) assert "TOO_MANY_DATABASES" in str(exp_info)
for i in range(10): for i in range(10):
node1.query("create table t{} (a Int32) Engine = Log".format(i)) node.query("create table t{} (a Int32) Engine = Log".format(i))
# This checks that system tables are not accounted in the number of tables.
node.query("system flush logs")
node1.query("system flush logs")
for i in range(10): for i in range(10):
node1.query("drop table t{}".format(i)) node.query("drop table t{}".format(i))
for i in range(10): for i in range(10):
node1.query("create table t{} (a Int32) Engine = Log".format(i)) node.query("create table t{} (a Int32) Engine = Log".format(i))
with pytest.raises(QueryRuntimeException) as exp_info: with pytest.raises(QueryRuntimeException) as exp_info:
node1.query("create table default.tx (a Int32) Engine = Log") node.query("create table default.tx (a Int32) Engine = Log")
assert "TOO_MANY_TABLES" in str(exp_info) assert "TOO_MANY_TABLES" in str(exp_info)

View File

@ -1,13 +1,36 @@
-- { echoOn }
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0 1 101 1 0 0 0
1 102 2 2 102 1 1 102 2 2 102 1
1 103 3 2 102 1 1 103 3 2 102 1
1 104 4 4 104 1 1 104 4 4 104 1
1 105 5 4 104 1 1 105 5 4 104 1
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0 1 101 1 0 0 0
1 102 2 2 102 1 1 102 2 2 102 1
1 103 3 2 102 1 1 103 3 2 102 1
1 104 4 4 104 1 1 104 4 4 104 1
1 105 5 4 104 1 1 105 5 4 104 1
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0
1 102 2 2 102 1
1 103 3 2 102 1
1 104 4 4 104 1
1 105 5 4 104 1
SET join_algorithm = 'full_sorting_merge';
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0
1 102 2 2 102 1
1 103 3 2 102 1
1 104 4 4 104 1
1 105 5 4 104 1
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0
1 102 2 2 102 1
1 103 3 2 102 1
1 104 4 4 104 1
1 105 5 4 104 1
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t);
1 101 1 0 0 0 1 101 1 0 0 0
1 102 2 2 102 1 1 102 2 2 102 1
1 103 3 2 102 1 1 103 3 2 102 1

View File

@ -4,20 +4,29 @@ DROP TABLE IF EXISTS B;
CREATE TABLE A(k UInt32, t UInt32, a UInt64) ENGINE = MergeTree() ORDER BY (k, t); CREATE TABLE A(k UInt32, t UInt32, a UInt64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO A(k,t,a) VALUES (1,101,1),(1,102,2),(1,103,3),(1,104,4),(1,105,5); INSERT INTO A(k,t,a) VALUES (1,101,1),(1,102,2),(1,103,3),(1,104,4),(1,105,5);
CREATE TABLE B(k UInt32, t UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); CREATE TABLE B1(k UInt32, t UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); INSERT INTO B1(k,t,b) VALUES (1,102,2), (1,104,4);
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t);
DROP TABLE B;
CREATE TABLE B2(t UInt32, k UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO B2(k,t,b) VALUES (1,102,2), (1,104,4);
CREATE TABLE B(t UInt32, k UInt32, b UInt64) ENGINE = MergeTree() ORDER BY (k, t); CREATE TABLE B3(k UInt32, b UInt64, t UInt32) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); INSERT INTO B3(k,t,b) VALUES (1,102,2), (1,104,4);
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t);
DROP TABLE B;
CREATE TABLE B(k UInt32, b UInt64, t UInt32) ENGINE = MergeTree() ORDER BY (k, t); -- { echoOn }
INSERT INTO B(k,t,b) VALUES (1,102,2), (1,104,4); SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t);
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t);
DROP TABLE B; SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t);
SET join_algorithm = 'full_sorting_merge';
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B1 B USING(k,t) ORDER BY (A.k, A.t);
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B2 B USING(k,t) ORDER BY (A.k, A.t);
SELECT A.k, A.t, A.a, B.b, B.t, B.k FROM A ASOF LEFT JOIN B3 B USING(k,t) ORDER BY (A.k, A.t);
-- { echoOff }
DROP TABLE B1;
DROP TABLE B2;
DROP TABLE B3;
DROP TABLE A; DROP TABLE A;

View File

@ -1 +1,2 @@
3000000 3000000
3000000

View File

@ -2,15 +2,28 @@
DROP TABLE IF EXISTS tvs; DROP TABLE IF EXISTS tvs;
-- to use different algorithms for in subquery
SET allow_experimental_analyzer = 1;
CREATE TABLE tvs(k UInt32, t UInt32, tv UInt64) ENGINE = Memory; CREATE TABLE tvs(k UInt32, t UInt32, tv UInt64) ENGINE = Memory;
INSERT INTO tvs(k,t,tv) SELECT k, t, t INSERT INTO tvs(k,t,tv) SELECT k, t, t
FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys
CROSS JOIN (SELECT toUInt32(number * 3) as t FROM numbers(10000)) tv_times; CROSS JOIN (SELECT toUInt32(number * 3) as t FROM numbers(10000)) tv_times
SETTINGS join_algorithm = 'hash';
SELECT SUM(trades.price - tvs.tv) FROM SELECT SUM(trades.price - tvs.tv) FROM
(SELECT k, t, t as price (SELECT k, t, t as price
FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys
CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times) trades CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times
SETTINGS join_algorithm = 'hash') trades
ASOF LEFT JOIN tvs USING(k,t); ASOF LEFT JOIN tvs USING(k,t);
SELECT SUM(trades.price - tvs.tv) FROM
(SELECT k, t, t as price
FROM (SELECT toUInt32(number) AS k FROM numbers(1000)) keys
CROSS JOIN (SELECT toUInt32(number * 10) AS t FROM numbers(3000)) trade_times
SETTINGS join_algorithm = 'hash') trades
ASOF LEFT JOIN tvs USING(k,t)
SETTINGS join_algorithm = 'full_sorting_merge';
DROP TABLE tvs; DROP TABLE tvs;

View File

@ -27,3 +27,32 @@
2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2 2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2 2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2 2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2
1 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0
1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1
1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1
2 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0
2 1970-01-01 00:00:02 2 0 1970-01-01 00:00:00 0
2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2
3 1970-01-01 00:00:01 1 0 1970-01-01 00:00:00 0
3 1970-01-01 00:00:02 2 0 1970-01-01 00:00:00 0
3 1970-01-01 00:00:03 3 0 1970-01-01 00:00:00 0
3 1970-01-01 00:00:04 4 0 1970-01-01 00:00:00 0
3 1970-01-01 00:00:05 5 0 1970-01-01 00:00:00 0
1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1
1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1
2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2
1 1970-01-01 00:00:02 2 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:03 3 2 1970-01-01 00:00:02 1
1 1970-01-01 00:00:04 4 4 1970-01-01 00:00:04 1
1 1970-01-01 00:00:05 5 4 1970-01-01 00:00:04 1
2 1970-01-01 00:00:03 3 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:04 4 3 1970-01-01 00:00:03 2
2 1970-01-01 00:00:05 5 3 1970-01-01 00:00:03 2

View File

@ -11,9 +11,12 @@ INSERT INTO B(k,t,b) VALUES (1,2,2),(1,4,4);
INSERT INTO B(k,t,b) VALUES (2,3,3); INSERT INTO B(k,t,b) VALUES (2,3,3);
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t);
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t);
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t);
SET join_algorithm = 'full_sorting_merge';
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (A.k, A.t);
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF INNER JOIN B ON A.k == B.k AND A.t >= B.t ORDER BY (A.k, A.t);
SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t); SELECT A.k, toString(A.t, 'UTC'), A.a, B.b, toString(B.t, 'UTC'), B.k FROM A ASOF JOIN B USING(k,t) ORDER BY (A.k, A.t);
DROP TABLE A; DROP TABLE A;

View File

@ -1,27 +1,72 @@
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1970-01-01 02:00:01 1 0 2 1970-01-01 02:00:01 1 0
2 1970-01-01 02:00:03 3 3 2 1970-01-01 02:00:03 3 3
2 1970-01-01 02:00:05 5 3 2 1970-01-01 02:00:05 5 3
-
2 1970-01-01 02:00:01 1 0
2 1970-01-01 02:00:03 3 3
2 1970-01-01 02:00:05 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0 2 1 1 0
2 3 3 3 2 3 3 3
2 5 5 3 2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1 1 0
2 3 3 3
2 5 5 3
-
2 1970-01-01 02:00:00.001 1 0
2 1970-01-01 02:00:00.003 3 3
2 1970-01-01 02:00:00.005 5 3
-
2 1970-01-01 02:00:00.001 1 0 2 1970-01-01 02:00:00.001 1 0
2 1970-01-01 02:00:00.003 3 3 2 1970-01-01 02:00:00.003 3 3
2 1970-01-01 02:00:00.005 5 3 2 1970-01-01 02:00:00.005 5 3

View File

@ -1,27 +0,0 @@
#!/usr/bin/env bash
set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
for typename in "UInt32" "UInt64" "Float64" "Float32" "DateTime('Asia/Istanbul')" "Decimal32(5)" "Decimal64(5)" "Decimal128(5)" "DateTime64(3, 'Asia/Istanbul')"
do
$CLICKHOUSE_CLIENT -mn <<EOF
DROP TABLE IF EXISTS A;
DROP TABLE IF EXISTS B;
CREATE TABLE A(k UInt32, t ${typename}, a Float64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO A(k,t,a) VALUES (2,1,1),(2,3,3),(2,5,5);
CREATE TABLE B(k UInt32, t ${typename}, b Float64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO B(k,t,b) VALUES (2,3,3);
SELECT k, t, a, b FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (k,t);
DROP TABLE A;
DROP TABLE B;
EOF
done

View File

@ -0,0 +1,27 @@
{% for typename in ["UInt32", "UInt64", "Float64", "Float32", "DateTime('Asia/Istanbul')", "Decimal32(5)", "Decimal64(5)", "Decimal128(5)", "DateTime64(3, 'Asia/Istanbul')"] -%}
DROP TABLE IF EXISTS A;
DROP TABLE IF EXISTS B;
CREATE TABLE A(k UInt32, t {{ typename }}, a Float64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO A(k,t,a) VALUES (2,1,1),(2,3,3),(2,5,5);
CREATE TABLE B(k UInt32, t {{ typename }}, b Float64) ENGINE = MergeTree() ORDER BY (k, t);
INSERT INTO B(k,t,b) VALUES (2,3,3);
SELECT '-';
SELECT k, t, a, b FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (k,t)
SETTINGS join_algorithm = 'full_sorting_merge';
SELECT '-';
SELECT k, t, a, b FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (k,t)
SETTINGS join_algorithm = 'hash';
DROP TABLE A;
DROP TABLE B;
{% endfor %}

View File

@ -12,3 +12,18 @@
2 1970-01-01 00:00:15 5 6.5 6 2 1970-01-01 00:00:15 5 6.5 6
2 1970-01-01 00:00:16 5 5.6 6 2 1970-01-01 00:00:16 5 5.6 6
2 1970-01-01 00:00:20 17 8.5 18 2 1970-01-01 00:00:20 17 8.5 18
-
1 1970-01-01 00:00:05 1 1.5 2
1 1970-01-01 00:00:06 1 1.51 2
1 1970-01-01 00:00:10 11 11.5 12
1 1970-01-01 00:00:11 11 11.51 12
1 1970-01-01 00:00:15 5 5.5 6
1 1970-01-01 00:00:16 5 5.6 6
1 1970-01-01 00:00:20 7 7.5 8
2 1970-01-01 00:00:05 11 2.5 12
2 1970-01-01 00:00:06 11 2.51 12
2 1970-01-01 00:00:10 21 12.5 22
2 1970-01-01 00:00:11 21 12.51 22
2 1970-01-01 00:00:15 5 6.5 6
2 1970-01-01 00:00:16 5 5.6 6
2 1970-01-01 00:00:20 17 8.5 18

View File

@ -9,7 +9,13 @@ CREATE TABLE tv(key UInt32, t DateTime, tv Float64) ENGINE = MergeTree() ORDER B
INSERT INTO tv(key,t,tv) VALUES (1,5,1.5),(1,6,1.51),(1,10,11.5),(1,11,11.51),(1,15,5.5),(1,16,5.6),(1,20,7.5); INSERT INTO tv(key,t,tv) VALUES (1,5,1.5),(1,6,1.51),(1,10,11.5),(1,11,11.51),(1,15,5.5),(1,16,5.6),(1,20,7.5);
INSERT INTO tv(key,t,tv) VALUES (2,5,2.5),(2,6,2.51),(2,10,12.5),(2,11,12.51),(2,15,6.5),(2,16,5.6),(2,20,8.5); INSERT INTO tv(key,t,tv) VALUES (2,5,2.5),(2,6,2.51),(2,10,12.5),(2,11,12.51),(2,15,6.5),(2,16,5.6),(2,20,8.5);
SELECT tv.key, toString(tv.t, 'UTC'), md.bid, tv.tv, md.ask FROM tv ASOF LEFT JOIN md USING(key,t) ORDER BY (tv.key, tv.t); SELECT tv.key, toString(tv.t, 'UTC'), md.bid, tv.tv, md.ask FROM tv ASOF LEFT JOIN md USING(key,t) ORDER BY (tv.key, tv.t)
;
SELECT '-';
SELECT tv.key, toString(tv.t, 'UTC'), md.bid, tv.tv, md.ask FROM tv ASOF LEFT JOIN md USING(key,t) ORDER BY (tv.key, tv.t)
SETTINGS join_algorithm = 'full_sorting_merge';
DROP TABLE md; DROP TABLE md;
DROP TABLE tv; DROP TABLE tv;

View File

@ -1,3 +1,4 @@
- default / join_use_nulls = 0 -
1 1 0 0 1 1 0 0
1 2 1 2 1 2 1 2
1 3 1 2 1 3 1 2
@ -34,3 +35,114 @@
2 1 2 3 2 1 2 3
2 2 2 3 2 2 2 3
1 2 1 2 1 2 1 2
- full_sorting_merge / join_use_nulls = 0 -
1 1 0 0
1 2 1 2
1 3 1 2
2 1 0 0
2 2 0 0
2 3 2 3
3 1 0 0
3 2 0 0
3 3 0 0
9
1 2 1 2
1 3 1 2
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 3 1 2
-
1 1 1 2
1 2 1 4
1 3 1 4
2 1 2 3
2 2 2 3
1 2 1 2
- default / join_use_nulls = 1 -
1 1 \N \N
1 2 1 2
1 3 1 2
2 1 \N \N
2 2 \N \N
2 3 2 3
3 1 \N \N
3 2 \N \N
3 3 \N \N
9
1 2 1 2
1 3 1 2
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 3 1 2
-
1 1 1 2
1 2 1 4
1 3 1 4
2 1 2 3
2 2 2 3
1 2 1 2
- full_sorting_merge / join_use_nulls = 1 -
1 1 \N \N
1 2 1 2
1 3 1 2
2 1 \N \N
2 2 \N \N
2 3 2 3
3 1 \N \N
3 2 \N \N
3 3 \N \N
9
1 2 1 2
1 3 1 2
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 1 1 2
1 2 1 2
1 3 1 4
2 1 2 3
2 2 2 3
2 3 2 3
-
1 3 1 2
-
1 1 1 2
1 2 1 4
1 3 1 4
2 1 2 3
2 2 2 3
1 2 1 2

View File

@ -7,6 +7,14 @@ CREATE TABLE B(b UInt32, t UInt32) ENGINE = Memory;
INSERT INTO A (a,t) VALUES (1,1),(1,2),(1,3), (2,1),(2,2),(2,3), (3,1),(3,2),(3,3); INSERT INTO A (a,t) VALUES (1,1),(1,2),(1,3), (2,1),(2,2),(2,3), (3,1),(3,2),(3,3);
INSERT INTO B (b,t) VALUES (1,2),(1,4),(2,3); INSERT INTO B (b,t) VALUES (1,2),(1,4),(2,3);
{% for join_use_nulls in [0, 1] -%}
{% for join_algorithm in ['default', 'full_sorting_merge'] -%}
SET join_algorithm = '{{ join_algorithm }}';
SELECT '- {{ join_algorithm }} / join_use_nulls = {{ join_use_nulls }} -';
set join_use_nulls = {{ join_use_nulls }};
SELECT A.a, A.t, B.b, B.t FROM A ASOF LEFT JOIN B ON A.a == B.b AND A.t >= B.t ORDER BY (A.a, A.t); SELECT A.a, A.t, B.b, B.t FROM A ASOF LEFT JOIN B ON A.a == B.b AND A.t >= B.t ORDER BY (A.a, A.t);
SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t; SELECT count() FROM A ASOF LEFT JOIN B ON A.a == B.b AND B.t <= A.t;
SELECT A.a, A.t, B.b, B.t FROM A ASOF INNER JOIN B ON B.t <= A.t AND A.a == B.b ORDER BY (A.a, A.t); SELECT A.a, A.t, B.b, B.t FROM A ASOF INNER JOIN B ON B.t <= A.t AND A.a == B.b ORDER BY (A.a, A.t);
@ -28,5 +36,8 @@ ASOF INNER JOIN (SELECT * FROM B UNION ALL SELECT 1, 3) AS B ON B.t <= A.t AND A
WHERE B.t != 3 ORDER BY (A.a, A.t) WHERE B.t != 3 ORDER BY (A.a, A.t)
; ;
{% endfor -%}
{% endfor -%}
DROP TABLE A; DROP TABLE A;
DROP TABLE B; DROP TABLE B;

View File

@ -1,3 +1,6 @@
v1 o1 ['s2','s1'] v1 o1 ['s2','s1']
v1 o2 ['s4'] v1 o2 ['s4']
v2 o3 ['s5','s3'] v2 o3 ['s5','s3']
v1 o1 ['s2','s1']
v1 o2 ['s4']
v2 o3 ['s5','s3']

View File

@ -16,3 +16,17 @@ GROUP BY
ORDER BY ORDER BY
visitorId ASC, visitorId ASC,
orderId ASC; orderId ASC;
SELECT
visitorId,
orderId,
groupUniqArray(sessionId)
FROM sessions
ASOF INNER JOIN orders ON (sessions.visitorId = orders.visitorId) AND (sessions.date <= orders.date)
GROUP BY
visitorId,
orderId
ORDER BY
visitorId ASC,
orderId ASC
SETTINGS join_algorithm = 'full_sorting_merge';

View File

@ -4,14 +4,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh . "$CURDIR"/../shell_config.sh
url="http://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/?session_id=test_01194" rnd="$CLICKHOUSE_DATABASE"
rnd=$RANDOM url="${CLICKHOUSE_URL}&session_id=test_01194_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+'test_01194',$rnd,1" > /dev/null ${CLICKHOUSE_CURL} -sS "$url&query=SELECT+'test_01194',$rnd,1" > /dev/null
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+'test_01194',$rnd,2" > /dev/null ${CLICKHOUSE_CURL} -sS "$url&query=SELECT+'test_01194',$rnd,2" > /dev/null
${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,3" > /dev/null ${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,3" > /dev/null
${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,4" > /dev/null ${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,4" > /dev/null
${CLICKHOUSE_CURL} -sS "$url" --data "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+count(DISTINCT+query_id)+FROM+system.query_log+WHERE+current_database+LIKE+currentDatabase()+AND+query+LIKE+'SELECT+''test_01194'',$rnd%25'" $CLICKHOUSE_CLIENT -q "
SELECT
count(DISTINCT query_id)
FROM system.query_log
WHERE
current_database = currentDatabase()
AND event_date >= yesterday()
AND query LIKE 'SELECT ''test_01194'',$rnd%'
AND query_id != queryID()"

View File

@ -1,3 +1,3 @@
10
5 5
CREATE TABLE default.alter_mt\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = MergeTree\nORDER BY key\nSETTINGS index_granularity = 8192 5
CREATE TABLE default.alter_mt\n(\n `key` Int64,\n `value` Int64\n)\nENGINE = MergeTree\nORDER BY key\nSETTINGS index_granularity = 8192

View File

@ -7,16 +7,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt"
$CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENGINE=MergeTree() ORDER BY key" $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key Int64, value String) ENGINE=MergeTree() ORDER BY key"
$CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number - 1 AS x, toString(x) FROM numbers(5)"
$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & $CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" &
# to be sure that select took all required locks # To be sure that select took all required locks for better test sensitivity, although it isn't guaranteed (then the test will also succeed).
sleep 2 sleep 2
$CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value UInt64" $CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value Int64"
$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt"

View File

@ -1,3 +1,3 @@
10
5 5
CREATE TABLE default.alter_mt\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01338_long_select_and_alter_zookeeper_default/alter_mt\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192 5
CREATE TABLE default.alter_mt\n(\n `key` Int64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01338_long_select_and_alter_zookeeper_default/alter_mt\', \'1\')\nORDER BY key\nSETTINGS index_granularity = 8192

View File

@ -7,16 +7,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS alter_mt"
$CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_mt', '1') ORDER BY key" $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key Int64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_mt', '1') ORDER BY key"
$CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number - 1 AS x, toString(x) FROM numbers(5)"
$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & $CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" &
# to be sure that select took all required locks # To be sure that select took all required locks for better test sensitivity, although it isn't guaranteed (then the test will also succeed).
sleep 2 sleep 2
$CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value UInt64" $CLICKHOUSE_CLIENT --query "ALTER TABLE alter_mt MODIFY COLUMN value Int64"
$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM alter_mt"

View File

@ -8,14 +8,14 @@ drop table if exists stack;
set max_insert_threads = 4; set max_insert_threads = 4;
create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64) create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64)
Engine = MergeTree Engine = MergeTree
partition by toYYYYMM(dt) partition by toYYYYMM(dt)
order by (brand_id, toStartOfHour(dt)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; order by (brand_id, toStartOfHour(dt)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into stack insert into stack
select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100, select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100,
toDateTime('2020-02-01 00:00:00')+number/10, intDiv(number,100)+1 toDateTime('2020-02-01 00:00:00')+number/10, intDiv(number,100)+1
from numbers_mt(10000000); from numbers_mt(1000000);
select '---- arrays ----'; select '---- arrays ----';
@ -32,8 +32,8 @@ select '---- window f ----';
select cityHash64( toString( groupArray (tuple(*) ) )) from ( select cityHash64( toString( groupArray (tuple(*) ) )) from (
select brand_id, rack_id, quantity from select brand_id, rack_id, quantity from
( select brand_id, rack_id, quantity, row_number() over (partition by brand_id, rack_id order by quantity) rn ( select brand_id, rack_id, quantity, row_number() over (partition by brand_id, rack_id order by quantity) rn
from stack ) as t0 from stack ) as t0
where rn <= 2 where rn <= 2
order by brand_id, rack_id, quantity order by brand_id, rack_id, quantity
) t; ) t;

View File

@ -2,3 +2,7 @@
0 340282366920938463463374607431768211457 0 340282366920938463463374607431768211457
0 18446744073709551617 0 18446744073709551617
0 340282366920938463463374607431768211457 0 340282366920938463463374607431768211457
0 18446744073709551617
0 340282366920938463463374607431768211457
0 18446744073709551617
0 340282366920938463463374607431768211457

View File

@ -3,3 +3,11 @@ select * from (select 0 as k, toInt256('340282366920938463463374607431768211457'
select * from (select 0 as k, toUInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toUInt128('18446744073709551616') as v) t2 using(k, v); select * from (select 0 as k, toUInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toUInt128('18446744073709551616') as v) t2 using(k, v);
select * from (select 0 as k, toUInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toUInt256('340282366920938463463374607431768211456') as v) t2 using(k, v); select * from (select 0 as k, toUInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toUInt256('340282366920938463463374607431768211456') as v) t2 using(k, v);
SET join_algorithm = 'full_sorting_merge';
select * from (select 0 as k, toInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toInt128('18446744073709551616') as v) t2 using(k, v);
select * from (select 0 as k, toInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toInt256('340282366920938463463374607431768211456') as v) t2 using(k, v);
select * from (select 0 as k, toUInt128('18446744073709551617') as v) t1 asof join (select 0 as k, toUInt128('18446744073709551616') as v) t2 using(k, v);
select * from (select 0 as k, toUInt256('340282366920938463463374607431768211457') as v) t1 asof join (select 0 as k, toUInt256('340282366920938463463374607431768211456') as v) t2 using(k, v);

View File

@ -10,59 +10,3 @@
1 1
1 1
1 1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -1,4 +1,4 @@
-- Tags: use-rocksdb, long, no-s3-storage, no-random-settings, no-random-merge-tree-settings -- Tags: use-rocksdb, long, no-s3-storage
SET join_algorithm = 'direct'; SET join_algorithm = 'direct';
@ -13,27 +13,21 @@ INSERT INTO rdb_{{ table_size }}
SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) AS value SELECT (sipHash64(number) % {{ table_size }}) as key, ('val' || toString(key)) AS value
FROM numbers_mt({{ table_size }}); FROM numbers_mt({{ table_size }});
{% for block_size in [10, 11, 128, 129, 65505, 65506, 70000] -%}
{% if block_size * 5000 > table_size -%}
SET max_block_size = {{ block_size }};
{% for right_size in [table_size // 2, table_size + table_size // 4 + 1] -%} {% for right_size in [table_size // 2, table_size + table_size // 4 + 1] -%}
SELECT count() == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }}) SELECT count() == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }})
FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1
INNER JOIN rdb_{{ table_size }} as rdb INNER JOIN rdb_{{ table_size }} as rdb
ON rdb.key == t1.k; ON rdb.key == t1.k
{% if table_size < 100 %}SETTINGS max_block_size = 1{% endif -%}
;
SELECT count() == {{ right_size }} and countIf(value != '') == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }}) SELECT count() == {{ right_size }} and countIf(value != '') == (SELECT count() FROM rdb_{{ table_size }} WHERE key < {{ right_size }})
FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1 FROM (SELECT number as k FROM numbers_mt({{ right_size }})) as t1
LEFT JOIN rdb_{{ table_size }} as rdb LEFT JOIN rdb_{{ table_size }} as rdb
ON rdb.key == t1.k; ON rdb.key == t1.k
{% if table_size < 100 %}SETTINGS max_block_size = 1{% endif -%}
{% endfor -%} ;
{% endif -%}
{% endfor -%} {% endfor -%}
{% endfor -%} {% endfor -%}

View File

@ -19,8 +19,6 @@ SELECT * FROM t1 ANTI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENT
SELECT * FROM t1 SEMI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 SEMI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED }
SELECT * FROM t1 ASOF JOIN t2 ON t1.key = t2.key AND t1.val > t2.val; -- { serverError NOT_IMPLEMENTED }
SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 ANY JOIN t2 ON t1.key = t2.key SETTINGS any_join_distinct_right_table_keys = 1; -- { serverError NOT_IMPLEMENTED }
SELECT * FROM t1 JOIN t2 USING (key) SETTINGS join_use_nulls = 1; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 JOIN t2 USING (key) SETTINGS join_use_nulls = 1; -- { serverError NOT_IMPLEMENTED }

View File

@ -6,3 +6,15 @@ ASOF LEFT JOIN (
select 1 as session_id, 4 as id select 1 as session_id, 4 as id
) as visitors ) as visitors
ON visitors.session_id <= sessions.id AND arrayFirst(a -> a, arrayMap((a) -> a, sessions.arr)) = visitors.id ON visitors.session_id <= sessions.id AND arrayFirst(a -> a, arrayMap((a) -> a, sessions.arr)) = visitors.id
;
select count(*)
from (
select 1 as id, [1, 2, 3] as arr
) as sessions
ASOF LEFT JOIN (
select 1 as session_id, 4 as id
) as visitors
ON visitors.session_id <= sessions.id AND arrayFirst(a -> a, arrayMap((a) -> a, sessions.arr)) = visitors.id
SETTINGS join_algorithm = 'full_sorting_merge'
;

View File

@ -35,6 +35,10 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N
2 2 2 2 2 2 2 2
3 3 3 33 3 3 3 33
\N \N \N \N \N \N \N \N
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
2 2 2 2
3 3 3 33
\N \N \N \N
-- aliases defined in the join condition are valid -- aliases defined in the join condition are valid
-- FIXME(@vdimir) broken query formatting for the following queries: -- FIXME(@vdimir) broken query formatting for the following queries:
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;

View File

@ -34,6 +34,7 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) A
SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
-- aliases defined in the join condition are valid -- aliases defined in the join condition are valid
-- FIXME(@vdimir) broken query formatting for the following queries: -- FIXME(@vdimir) broken query formatting for the following queries:

View File

@ -1,10 +0,0 @@
before
rmt_master NewPart 0 1
rmt_master MergeParts 0 1
rmt_slave MergeParts 1 0
rmt_slave DownloadPart 0 1
after
rmt_master NewPart 0 1
rmt_master MergeParts 0 1
rmt_slave MergeParts 1 0
rmt_slave DownloadPart 0 2

View File

@ -1,35 +0,0 @@
-- Tags: no-replicated-database, no-parallel, no-shared-merge-tree
-- SMT: The merge process is completely different from RMT
drop table if exists rmt_master;
drop table if exists rmt_slave;
create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by key settings always_fetch_merged_part=0;
-- always_fetch_merged_part=1, consider this table as a "slave"
create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by key settings always_fetch_merged_part=1;
insert into rmt_master values (1);
system sync replica rmt_master;
system sync replica rmt_slave;
system stop replicated sends rmt_master;
optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1;
select sleep(3) format Null;
system flush logs;
select 'before';
select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3;
system start replicated sends rmt_master;
-- sleep few seconds to try rmt_slave to fetch the part and reflect this error
-- in system.part_log
select sleep(3) format Null;
system sync replica rmt_slave;
system flush logs;
select 'after';
select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3;
drop table rmt_master;
drop table rmt_slave;

View File

@ -1,10 +0,0 @@
before
rmt_master NewPart 0 1
rmt_master MutatePart 0 1
rmt_slave DownloadPart 0 1
rmt_slave MutatePart 1 0
after
rmt_master NewPart 0 1
rmt_master MutatePart 0 1
rmt_slave DownloadPart 0 2
rmt_slave MutatePart 1 0

View File

@ -1,41 +0,0 @@
-- Tags: no-replicated-database, no-parallel, no-shared-merge-tree
-- SMT: The merge process is completely different from RMT
drop table if exists rmt_master;
drop table if exists rmt_slave;
create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by tuple() settings always_fetch_merged_part=0, old_parts_lifetime=600;
-- prefer_fetch_merged_part_*_threshold=0, consider this table as a "slave"
create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by tuple() settings prefer_fetch_merged_part_time_threshold=0, prefer_fetch_merged_part_size_threshold=0, old_parts_lifetime=600;
insert into rmt_master values (1);
system sync replica rmt_master;
system sync replica rmt_slave;
system stop replicated sends rmt_master;
system stop pulling replication log rmt_slave;
alter table rmt_master update key=key+100 where 1 settings alter_sync=1;
-- first we need to make the rmt_master execute mutation so that it will have
-- the part, and rmt_slave will consider it instead of performing mutation on
-- it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored
select sleep(3) format Null;
system start pulling replication log rmt_slave;
-- and sleep few more seconds to try rmt_slave to fetch the part and reflect
-- this error in system.part_log
select sleep(3) format Null;
system flush logs;
select 'before';
select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3;
system start replicated sends rmt_master;
select sleep(3) format Null;
system sync replica rmt_slave;
system flush logs;
select 'after';
select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3;
drop table rmt_master;
drop table rmt_slave;

View File

@ -0,0 +1,2 @@
49999983751397 10000032
49999983751397 10000032

View File

@ -0,0 +1,50 @@
-- Tags: long
DROP TABLE IF EXISTS build;
DROP TABLE IF EXISTS skewed_probe;
SET session_timezone = 'UTC';
CREATE TABLE build ENGINE = MergeTree ORDER BY (key, begin)
AS
SELECT
toDateTime('1990-03-21 13:00:00') + INTERVAL number MINUTE AS begin,
number % 4 AS key,
number AS value
FROM numbers(0, 10000000);
CREATE TABLE skewed_probe ENGINE = MergeTree ORDER BY (key, begin)
AS
SELECT
toDateTime('1990-04-21 13:00:01') + INTERVAL number MINUTE AS begin,
0 AS key
FROM numbers(0, 5)
UNION ALL
SELECT
toDateTime('1990-05-21 13:00:01') + INTERVAL number MINUTE AS begin,
1 AS key
FROM numbers(0, 10)
UNION ALL
SELECT
toDateTime('1990-06-21 13:00:01') + INTERVAL number MINUTE AS begin,
2 AS key
FROM numbers(0, 20)
UNION ALL
SELECT
toDateTime('1990-03-21 13:00:01') + INTERVAL number MINUTE AS begin,
3 AS key
FROM numbers(0, 10000000);
SELECT SUM(value), COUNT(*)
FROM skewed_probe
ASOF JOIN build
USING (key, begin)
;
SELECT SUM(value), COUNT(*)
FROM skewed_probe
ASOF JOIN build
USING (key, begin)
SETTINGS join_algorithm = 'full_sorting_merge'
;

View File

@ -0,0 +1,58 @@
1 0
2 0
3 1
4 1
5 1
6 2
7 2
8 3
9 3
0 0
1 0
2 0
3 1
4 1
5 1
6 2
7 2
8 3
9 3
1 1 0
1 2 0
1 3 1
1 4 1
1 5 1
1 6 2
1 7 2
1 8 3
1 9 3
2 0 10
2 1 10
2 2 10
2 3 10
2 4 10
2 5 10
2 6 10
2 7 20
2 8 20
2 9 20
1 0 0
1 1 0
1 2 0
1 3 1
1 4 1
1 5 1
1 6 2
1 7 2
1 8 3
1 9 3
2 0 10
2 1 10
2 2 10
2 3 10
2 4 10
2 5 10
2 6 10
2 7 20
2 8 20
2 9 20

View File

@ -0,0 +1,65 @@
SET join_algorithm = 'full_sorting_merge';
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS events0;
CREATE TABLE events0 (
begin Float64,
value Int32
) ENGINE = MergeTree ORDER BY begin;
INSERT INTO events0 VALUES (1.0, 0), (3.0, 1), (6.0, 2), (8.0, 3);
SELECT p.ts, e.value
FROM
(SELECT number :: Float64 AS ts FROM numbers(10)) p
ASOF JOIN events0 e
ON p.ts >= e.begin
ORDER BY p.ts ASC;
SELECT p.ts, e.value
FROM
(SELECT number :: Float64 AS ts FROM numbers(10)) p
ASOF LEFT JOIN events0 e
ON p.ts >= e.begin
ORDER BY p.ts ASC
-- SETTINGS join_use_nulls = 1
;
DROP TABLE IF EXISTS events0;
DROP TABLE IF EXISTS events;
DROP TABLE IF EXISTS probes;
CREATE TABLE events (
key Int32,
begin Float64,
value Int32
) ENGINE = MergeTree ORDER BY (key, begin);
INSERT INTO events VALUES (1, 1.0, 0), (1, 3.0, 1), (1, 6.0, 2), (1, 8.0, 3), (2, 0.0, 10), (2, 7.0, 20), (2, 11.0, 30);
CREATE TABLE probes (
key Int32,
ts Float64
) ENGINE = MergeTree ORDER BY (key, ts) AS
SELECT
key.number,
ts.number
FROM
numbers(1, 2) as key,
numbers(10) as ts
SETTINGS join_algorithm = 'hash';
SELECT p.key, p.ts, e.value
FROM probes p
ASOF JOIN events e
ON p.key = e.key AND p.ts >= e.begin
ORDER BY p.key, p.ts ASC;
SELECT p.key, p.ts, e.value
FROM probes p
ASOF LEFT JOIN events e
ON p.key = e.key AND p.ts >= e.begin
ORDER BY p.key, p.ts ASC NULLS FIRST;

View File

@ -0,0 +1,73 @@
-
2023-03-21 12:00:00 1970-01-01 00:00:00 -1
2023-03-21 13:00:00 1970-01-01 00:00:00 -1
2023-03-21 14:00:00 2023-03-21 13:00:00 0
2023-03-21 15:00:00 2023-03-21 14:00:00 1
2023-03-21 16:00:00 2023-03-21 15:00:00 2
2023-03-21 17:00:00 2023-03-21 16:00:00 3
2023-03-21 18:00:00 2023-03-21 16:00:00 3
2023-03-21 19:00:00 2023-03-21 16:00:00 3
2023-03-21 20:00:00 2023-03-21 16:00:00 3
2023-03-21 21:00:00 2023-03-21 16:00:00 3
2027-10-18 11:03:27 2023-03-21 16:00:00 3
-
2023-03-21 12:00:00 1970-01-01 00:00:00 -1
2023-03-21 13:00:00 1970-01-01 00:00:00 -1
2023-03-21 14:00:00 2023-03-21 13:00:00 0
2023-03-21 15:00:00 2023-03-21 14:00:00 1
2023-03-21 16:00:00 2023-03-21 15:00:00 2
2023-03-21 17:00:00 2023-03-21 16:00:00 3
2023-03-21 18:00:00 2023-03-21 16:00:00 3
2023-03-21 19:00:00 2023-03-21 16:00:00 3
2023-03-21 20:00:00 2023-03-21 16:00:00 3
2023-03-21 21:00:00 2023-03-21 16:00:00 3
2027-10-18 11:03:27 2023-03-21 16:00:00 3
\N \N \N
2023-03-21 12:00:00 2023-03-21 13:00:00 0
2023-03-21 13:00:00 2023-03-21 13:00:00 0
2023-03-21 14:00:00 2023-03-21 14:00:00 1
2023-03-21 15:00:00 2023-03-21 15:00:00 2
2023-03-21 16:00:00 2023-03-21 16:00:00 3
2023-03-21 17:00:00 2027-10-18 11:03:27 9
2023-03-21 18:00:00 2027-10-18 11:03:27 9
2023-03-21 19:00:00 2027-10-18 11:03:27 9
2023-03-21 20:00:00 2027-10-18 11:03:27 9
2023-03-21 21:00:00 2027-10-18 11:03:27 9
2027-10-18 11:03:27 2027-10-18 11:03:27 9
-
2023-03-21 12:00:00 2023-03-21 13:00:00 0
2023-03-21 13:00:00 2023-03-21 13:00:00 0
2023-03-21 14:00:00 2023-03-21 14:00:00 1
2023-03-21 15:00:00 2023-03-21 15:00:00 2
2023-03-21 16:00:00 2023-03-21 16:00:00 3
2023-03-21 17:00:00 2027-10-18 11:03:27 9
2023-03-21 18:00:00 2027-10-18 11:03:27 9
2023-03-21 19:00:00 2027-10-18 11:03:27 9
2023-03-21 20:00:00 2027-10-18 11:03:27 9
2023-03-21 21:00:00 2027-10-18 11:03:27 9
2027-10-18 11:03:27 2027-10-18 11:03:27 9
\N \N \N
-
2023-03-21 12:00:00 2023-03-21 13:00:00 0
2023-03-21 13:00:00 2023-03-21 14:00:00 1
2023-03-21 14:00:00 2023-03-21 15:00:00 2
2023-03-21 15:00:00 2023-03-21 16:00:00 3
2023-03-21 16:00:00 2027-10-18 11:03:27 9
2023-03-21 17:00:00 2027-10-18 11:03:27 9
2023-03-21 18:00:00 2027-10-18 11:03:27 9
2023-03-21 19:00:00 2027-10-18 11:03:27 9
2023-03-21 20:00:00 2027-10-18 11:03:27 9
2023-03-21 21:00:00 2027-10-18 11:03:27 9
-
2023-03-21 12:00:00 2023-03-21 13:00:00 0
2023-03-21 13:00:00 2023-03-21 14:00:00 1
2023-03-21 14:00:00 2023-03-21 15:00:00 2
2023-03-21 15:00:00 2023-03-21 16:00:00 3
2023-03-21 16:00:00 2027-10-18 11:03:27 9
2023-03-21 17:00:00 2027-10-18 11:03:27 9
2023-03-21 18:00:00 2027-10-18 11:03:27 9
2023-03-21 19:00:00 2027-10-18 11:03:27 9
2023-03-21 20:00:00 2027-10-18 11:03:27 9
2023-03-21 21:00:00 2027-10-18 11:03:27 9
2027-10-18 11:03:27 \N \N
\N \N \N

View File

@ -0,0 +1,66 @@
DROP TABLE IF EXISTS events0;
DROP TABLE IF EXISTS probe0;
SET allow_experimental_analyzer = 1;
SET join_algorithm = 'full_sorting_merge';
CREATE TABLE events0 (
begin Nullable(DateTime('UTC')),
value Int32
) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO events0 SELECT toDateTime('2023-03-21 13:00:00', 'UTC') + INTERVAL number HOUR, number FROM numbers(4);
INSERT INTO events0 VALUES (NULL, -10),('0000-01-01 00:00:00', -1), ('9999-12-31 23:59:59', 9);
CREATE TABLE probe0 (
begin Nullable(DateTime('UTC'))
) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO probe0 SELECT toDateTime('2023-03-21 12:00:00', 'UTC') + INTERVAl number HOUR FROM numbers(10);
INSERT INTO probe0 VALUES (NULL),('9999-12-31 23:59:59');
SET join_use_nulls = 1;
SELECT '-';
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF JOIN events0 e
ON p.begin > e.begin
ORDER BY p.begin ASC;
SELECT '-';
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF LEFT JOIN events0 e
ON p.begin > e.begin
ORDER BY p.begin ASC;
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF JOIN events0 e
ON p.begin <= e.begin
ORDER BY p.begin ASC;
SELECT '-';
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF LEFT JOIN events0 e
ON p.begin <= e.begin
ORDER BY p.begin ASC;
SELECT '-';
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF JOIN events0 e
ON p.begin < e.begin
ORDER BY p.begin ASC;
SELECT '-';
SELECT p.begin, e.begin, e.value
FROM probe0 p
ASOF LEFT JOIN events0 e
ON p.begin < e.begin
ORDER BY p.begin ASC;
DROP TABLE IF EXISTS events0;
DROP TABLE IF EXISTS probe0;

View File

@ -0,0 +1,2 @@
26790 1488
26790 1488

View File

@ -0,0 +1,39 @@
-- Tags: long
SET allow_experimental_analyzer=1;
SET session_timezone = 'UTC';
{% for join_algorithm in ['default', 'full_sorting_merge'] -%}
SET join_algorithm = '{{ join_algorithm }}';
-- TODO: enable once USING and `join_use_nulls` is supported by `full_sorting_merge`
-- SET join_use_nulls = 1;
WITH build AS (
SELECT
tk.number AS k,
toDateTime('2021-01-01 00:00:00') + INTERVAL i.number SECONDS AS t,
i.number % 37 AS v
FROM numbers(3000000) AS i
CROSS JOIN numbers(2) AS tk
SETTINGS join_algorithm = 'hash', join_use_nulls = 0
),
probe AS (
SELECT
tk.number AS k,
toDateTime('2021-01-01 00:00:30') + INTERVAL tt.number HOUR AS t
FROM numbers(2) AS tk
CROSS JOIN numbers(toUInt32((toDateTime('2021-02-01 00:00:30') - toDateTime('2021-01-01 00:00:30')) / 3600)) AS tt
SETTINGS join_algorithm = 'hash', join_use_nulls = 0
)
SELECT
SUM(v) AS v,
COUNT(*) AS n
FROM probe
ASOF LEFT JOIN build
USING (k, t)
;
{% endfor -%}

View File

@ -0,0 +1,10 @@
108
108 27
513
1218
3528
14553
121275
1495503
12462525
1249625025

View File

@ -0,0 +1,186 @@
SET allow_experimental_analyzer=1;
SET session_timezone = 'UTC';
SET joined_subquery_requires_alias = 0;
SET allow_experimental_analyzer = 1;
SET join_algorithm = 'full_sorting_merge';
-- # 10 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(10), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # Coverage: Missing right side bin
WITH build AS (
SELECT
k * 2 AS k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(10), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
intDiv(k, 2) AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v), COUNT(*)
FROM probe ASOF JOIN build USING (k, t);
-- # 20 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(20), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 30 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(30), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 50 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(50), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 100 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(100), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 100 dates, 50 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(100), (SELECT number AS k FROM numbers(50))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 1000 dates, 5 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(1000), (SELECT number AS k FROM numbers(5))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 1000 dates, 50 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(1000), (SELECT number AS k FROM numbers(50))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);
-- # 10000 dates, 50 keys
WITH build AS (
SELECT
k,
toDateTime('2001-01-01 00:00:00') + INTERVAL number MINUTE AS t,
number AS v
FROM numbers(10000), (SELECT number AS k FROM numbers(50))
SETTINGS join_algorithm = 'default'
),
probe AS (
SELECT
k * 2 AS k,
t - INTERVAL 30 SECOND AS t
FROM build
)
SELECT SUM(v)
FROM probe ASOF JOIN build USING (k, t);

View File

@ -0,0 +1,4 @@
1 1
3 1
6 1
8 1

View File

@ -0,0 +1,29 @@
DROP TABLE IF EXISTS events;
CREATE TABLE events (begin Float64, value Int32) ENGINE = MergeTree() ORDER BY begin;
INSERT INTO events VALUES (1, 0), (3, 1), (6, 2), (8, 3);
SET allow_experimental_analyzer = 1;
SET join_algorithm = 'full_sorting_merge';
SET joined_subquery_requires_alias = 0;
SELECT
begin,
value IN (
SELECT e1.value
FROM (
SELECT *
FROM events e1
WHERE e1.value = events.value
) AS e1
ASOF JOIN (
SELECT number :: Float64 AS begin
FROM numbers(10)
WHERE number >= 1 AND number < 10
)
USING (begin)
)
FROM events
ORDER BY begin ASC;
DROP TABLE IF EXISTS events;

View File

@ -0,0 +1,56 @@
2023-03-21 13:00:00 0
2023-03-21 14:00:00 1
2023-03-21 15:00:00 2
2023-03-21 16:00:00 3
2023-03-21 17:00:00 3
2023-03-21 18:00:00 3
2023-03-21 19:00:00 3
2023-03-21 20:00:00 3
2023-03-21 21:00:00 3
2106-02-07 06:28:15 9
2023-03-21 13:00:00 0
2023-03-21 14:00:00 1
2023-03-21 15:00:00 2
2023-03-21 16:00:00 3
2023-03-21 17:00:00 3
2023-03-21 18:00:00 3
2023-03-21 19:00:00 3
2023-03-21 20:00:00 3
2023-03-21 21:00:00 3
2106-02-07 06:28:15 9
2023-03-21 12:00:00 \N
2023-03-21 13:00:00 0
2023-03-21 14:00:00 1
2023-03-21 15:00:00 2
2023-03-21 16:00:00 3
2023-03-21 17:00:00 3
2023-03-21 18:00:00 3
2023-03-21 19:00:00 3
2023-03-21 20:00:00 3
2023-03-21 21:00:00 3
2106-02-07 06:28:15 9
\N \N
2023-03-21 12:00:00 0
2023-03-21 13:00:00 0
2023-03-21 14:00:00 1
2023-03-21 15:00:00 2
2023-03-21 16:00:00 3
2023-03-21 17:00:00 3
2023-03-21 18:00:00 3
2023-03-21 19:00:00 3
2023-03-21 20:00:00 3
2023-03-21 21:00:00 3
2106-02-07 06:28:15 9
\N 0
2023-03-21 12:00:00 \N
2023-03-21 13:00:00 \N
2023-03-21 14:00:00 \N
2023-03-21 15:00:00 \N
2023-03-21 16:00:00 \N
2023-03-21 17:00:00 \N
2023-03-21 18:00:00 \N
2023-03-21 19:00:00 \N
2023-03-21 20:00:00 \N
2023-03-21 21:00:00 \N
2106-02-07 06:28:15 \N
\N \N

View File

@ -0,0 +1,95 @@
DROP TABLE IF EXISTS events0;
DROP TABLE IF EXISTS probe0;
SET session_timezone = 'UTC';
SET allow_experimental_analyzer = 1;
SET join_algorithm = 'full_sorting_merge';
SET join_use_nulls = 1;
CREATE TABLE events0
ENGINE = MergeTree()
ORDER BY COALESCE(begin, toDateTime('9999-12-31 23:59:59'))
AS
SELECT
toNullable(toDateTime('2023-03-21 13:00:00') + INTERVAL number HOUR) AS begin,
number AS value
FROM numbers(4);
INSERT INTO events0 VALUES (NULL, -1), (toDateTime('9999-12-31 23:59:59'), 9);
CREATE TABLE probe0
ENGINE = MergeTree()
ORDER BY COALESCE(begin, toDateTime('9999-12-31 23:59:59'))
AS
SELECT
toNullable(toDateTime('2023-03-21 12:00:00') + INTERVAL number HOUR) AS begin
FROM numbers(10);
INSERT INTO probe0 VALUES (NULL), (toDateTime('9999-12-31 23:59:59'));
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF JOIN events0 e ON p.begin >= e.begin
ORDER BY p.begin ASC;
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF JOIN events0 e USING (begin)
ORDER BY p.begin ASC
SETTINGS join_use_nulls = 0
;
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF LEFT JOIN events0 e ON p.begin >= e.begin
ORDER BY p.begin ASC;
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF LEFT JOIN events0 e USING (begin)
ORDER BY p.begin ASC
SETTINGS join_use_nulls = 0
;
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF RIGHT JOIN events0 e ON p.begin >= e.begin
ORDER BY e.begin ASC; -- { serverError NOT_IMPLEMENTED}
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF RIGHT JOIN events0 e USING (begin)
ORDER BY e.begin ASC; -- { serverError NOT_IMPLEMENTED}
SELECT
p.begin,
e.value
FROM
probe0 p
ASOF LEFT JOIN (
SELECT * FROM events0 WHERE log(value + 5) > 10
) e ON p.begin + INTERVAL 2 HOUR >= e.begin + INTERVAL 1 HOUR
ORDER BY p.begin ASC;
DROP TABLE IF EXISTS events0;
DROP TABLE IF EXISTS probe0;