mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Compare commits
31 Commits
cb5fc34f29
...
3fa9ca9f1e
Author | SHA1 | Date | |
---|---|---|---|
|
3fa9ca9f1e | ||
|
7eee149487 | ||
|
cc3c7e74ae | ||
|
e0f8b8d351 | ||
|
72d5af29e0 | ||
|
287aeba8eb | ||
|
85631dba54 | ||
|
0a2b97c9c6 | ||
|
1bbfe5340a | ||
|
3fcd892a36 | ||
|
410951d64f | ||
|
f45bd58849 | ||
|
57db5cf24c | ||
|
459fa898ed | ||
|
0d875ecf5c | ||
|
6698212b5a | ||
|
c787838cb2 | ||
|
eb020f1c4b | ||
|
1a40df4d0c | ||
|
4380c6035d | ||
|
5145281088 | ||
|
35fa4c43e4 | ||
|
293e076493 | ||
|
8b92603c6d | ||
|
fb14f6e029 | ||
|
e1f37ec2bb | ||
|
cc0ef6104f | ||
|
46ce65e66e | ||
|
e048893b85 | ||
|
3827d90bb0 | ||
|
bf3a3ad607 |
@ -341,7 +341,10 @@ Coordination::Error ZooKeeper::tryGetChildren(
|
|||||||
const EventPtr & watch,
|
const EventPtr & watch,
|
||||||
Coordination::ListRequestType list_request_type)
|
Coordination::ListRequestType list_request_type)
|
||||||
{
|
{
|
||||||
return tryGetChildrenWatch(path, res, stat,
|
return tryGetChildrenWatch(
|
||||||
|
path,
|
||||||
|
res,
|
||||||
|
stat,
|
||||||
watch ? std::make_shared<Coordination::WatchCallback>(callbackForEvent(watch)) : Coordination::WatchCallbackPtr{},
|
watch ? std::make_shared<Coordination::WatchCallback>(callbackForEvent(watch)) : Coordination::WatchCallbackPtr{},
|
||||||
list_request_type);
|
list_request_type);
|
||||||
}
|
}
|
||||||
@ -975,11 +978,14 @@ void ZooKeeper::removeRecursive(const std::string & path, uint32_t remove_nodes_
|
|||||||
|
|
||||||
Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit)
|
Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit)
|
||||||
{
|
{
|
||||||
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
|
const auto fallback_method = [&]
|
||||||
{
|
{
|
||||||
tryRemoveChildrenRecursive(path);
|
tryRemoveChildrenRecursive(path);
|
||||||
return tryRemove(path);
|
return tryRemove(path);
|
||||||
}
|
};
|
||||||
|
|
||||||
|
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
|
||||||
|
return fallback_method();
|
||||||
|
|
||||||
auto promise = std::make_shared<std::promise<Coordination::RemoveRecursiveResponse>>();
|
auto promise = std::make_shared<std::promise<Coordination::RemoveRecursiveResponse>>();
|
||||||
auto future = promise->get_future();
|
auto future = promise->get_future();
|
||||||
@ -998,6 +1004,10 @@ Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint
|
|||||||
}
|
}
|
||||||
|
|
||||||
auto response = future.get();
|
auto response = future.get();
|
||||||
|
|
||||||
|
if (response.error == Coordination::Error::ZNOTEMPTY) /// limit was too low, try without RemoveRecursive request
|
||||||
|
return fallback_method();
|
||||||
|
|
||||||
return response.error;
|
return response.error;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -486,13 +486,13 @@ public:
|
|||||||
/// Remove the node with the subtree.
|
/// Remove the node with the subtree.
|
||||||
/// If Keeper supports RemoveRecursive operation then it will be performed atomically.
|
/// If Keeper supports RemoveRecursive operation then it will be performed atomically.
|
||||||
/// Otherwise if someone concurrently adds or removes a node in the subtree, the result is undefined.
|
/// Otherwise if someone concurrently adds or removes a node in the subtree, the result is undefined.
|
||||||
void removeRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
|
void removeRecursive(const std::string & path, uint32_t remove_nodes_limit = 1000);
|
||||||
|
|
||||||
/// Same as removeRecursive but in case if Keeper does not supports RemoveRecursive and
|
/// Same as removeRecursive but in case if Keeper does not supports RemoveRecursive and
|
||||||
/// if someone concurrently removes a node in the subtree, this will not cause errors.
|
/// if someone concurrently removes a node in the subtree, this will not cause errors.
|
||||||
/// For instance, you can call this method twice concurrently for the same node and the end
|
/// For instance, you can call this method twice concurrently for the same node and the end
|
||||||
/// result would be the same as for the single call.
|
/// result would be the same as for the single call.
|
||||||
Coordination::Error tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
|
Coordination::Error tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit = 1000);
|
||||||
|
|
||||||
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
|
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
|
||||||
/// Node defined as RemoveException will not be deleted.
|
/// Node defined as RemoveException will not be deleted.
|
||||||
|
@ -767,6 +767,11 @@ size_t ZooKeeperMultiRequest::sizeImpl() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
void ZooKeeperMultiRequest::readImpl(ReadBuffer & in)
|
void ZooKeeperMultiRequest::readImpl(ReadBuffer & in)
|
||||||
|
{
|
||||||
|
readImpl(in, /*request_validator=*/{});
|
||||||
|
}
|
||||||
|
|
||||||
|
void ZooKeeperMultiRequest::readImpl(ReadBuffer & in, RequestValidator request_validator)
|
||||||
{
|
{
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
@ -788,6 +793,8 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in)
|
|||||||
|
|
||||||
ZooKeeperRequestPtr request = ZooKeeperRequestFactory::instance().get(op_num);
|
ZooKeeperRequestPtr request = ZooKeeperRequestFactory::instance().get(op_num);
|
||||||
request->readImpl(in);
|
request->readImpl(in);
|
||||||
|
if (request_validator)
|
||||||
|
request_validator(*request);
|
||||||
requests.push_back(request);
|
requests.push_back(request);
|
||||||
|
|
||||||
if (in.eof())
|
if (in.eof())
|
||||||
|
@ -570,6 +570,9 @@ struct ZooKeeperMultiRequest final : MultiRequest<ZooKeeperRequestPtr>, ZooKeepe
|
|||||||
void writeImpl(WriteBuffer & out) const override;
|
void writeImpl(WriteBuffer & out) const override;
|
||||||
size_t sizeImpl() const override;
|
size_t sizeImpl() const override;
|
||||||
void readImpl(ReadBuffer & in) override;
|
void readImpl(ReadBuffer & in) override;
|
||||||
|
|
||||||
|
using RequestValidator = std::function<void(const ZooKeeperRequest &)>;
|
||||||
|
void readImpl(ReadBuffer & in, RequestValidator request_validator);
|
||||||
std::string toStringImpl(bool short_format) const override;
|
std::string toStringImpl(bool short_format) const override;
|
||||||
|
|
||||||
ZooKeeperResponsePtr makeResponse() const override;
|
ZooKeeperResponsePtr makeResponse() const override;
|
||||||
|
@ -514,7 +514,13 @@ void KeeperContext::initializeFeatureFlags(const Poco::Util::AbstractConfigurati
|
|||||||
feature_flags.disableFeatureFlag(feature_flag.value());
|
feature_flags.disableFeatureFlag(feature_flag.value());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ))
|
||||||
|
feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST);
|
||||||
|
else
|
||||||
|
system_nodes_with_data[keeper_api_version_path] = toString(static_cast<uint8_t>(KeeperApiVersion::ZOOKEEPER_COMPATIBLE));
|
||||||
|
|
||||||
system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags();
|
system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
feature_flags.logFlags(getLogger("KeeperContext"));
|
feature_flags.logFlags(getLogger("KeeperContext"));
|
||||||
@ -569,6 +575,25 @@ const CoordinationSettings & KeeperContext::getCoordinationSettings() const
|
|||||||
return *coordination_settings;
|
return *coordination_settings;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool KeeperContext::isOperationSupported(Coordination::OpNum operation) const
|
||||||
|
{
|
||||||
|
switch (operation)
|
||||||
|
{
|
||||||
|
case Coordination::OpNum::FilteredList:
|
||||||
|
return feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST);
|
||||||
|
case Coordination::OpNum::MultiRead:
|
||||||
|
return feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ);
|
||||||
|
case Coordination::OpNum::CreateIfNotExists:
|
||||||
|
return feature_flags.isEnabled(KeeperFeatureFlag::CREATE_IF_NOT_EXISTS);
|
||||||
|
case Coordination::OpNum::CheckNotExists:
|
||||||
|
return feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS);
|
||||||
|
case Coordination::OpNum::RemoveRecursive:
|
||||||
|
return feature_flags.isEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE);
|
||||||
|
default:
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
uint64_t KeeperContext::lastCommittedIndex() const
|
uint64_t KeeperContext::lastCommittedIndex() const
|
||||||
{
|
{
|
||||||
return last_committed_log_idx.load(std::memory_order_relaxed);
|
return last_committed_log_idx.load(std::memory_order_relaxed);
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <Coordination/KeeperFeatureFlags.h>
|
#include <Coordination/KeeperFeatureFlags.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include <condition_variable>
|
#include <condition_variable>
|
||||||
#include <cstdint>
|
#include <cstdint>
|
||||||
@ -103,6 +104,7 @@ public:
|
|||||||
return precommit_sleep_probability_for_testing;
|
return precommit_sleep_probability_for_testing;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool isOperationSupported(Coordination::OpNum operation) const;
|
||||||
private:
|
private:
|
||||||
/// local disk defined using path or disk name
|
/// local disk defined using path or disk name
|
||||||
using Storage = std::variant<DiskPtr, std::string>;
|
using Storage = std::variant<DiskPtr, std::string>;
|
||||||
|
@ -206,10 +206,12 @@ namespace DB
|
|||||||
DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \
|
DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \
|
||||||
DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \
|
DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \
|
||||||
DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \
|
DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \
|
||||||
|
DECLARE(UInt64, startup_mv_delay_ms, 0, "Debug parameter to simulate materizlied view creation delay", 0) \
|
||||||
|
|
||||||
|
|
||||||
// clang-format on
|
// clang-format on
|
||||||
|
|
||||||
|
|
||||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
|
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
|
||||||
|
|
||||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||||
|
@ -209,6 +209,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
|
|||||||
{
|
{
|
||||||
size_t prev_tables_count = metadata.parsed_tables.size();
|
size_t prev_tables_count = metadata.parsed_tables.size();
|
||||||
size_t prev_total_dictionaries = metadata.total_dictionaries;
|
size_t prev_total_dictionaries = metadata.total_dictionaries;
|
||||||
|
size_t prev_total_materialized_views = metadata.total_materialized_views;
|
||||||
|
|
||||||
auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name)
|
auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name)
|
||||||
{
|
{
|
||||||
@ -276,6 +277,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
|
|||||||
std::lock_guard lock{metadata.mutex};
|
std::lock_guard lock{metadata.mutex};
|
||||||
metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast};
|
metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast};
|
||||||
metadata.total_dictionaries += create_query->is_dictionary;
|
metadata.total_dictionaries += create_query->is_dictionary;
|
||||||
|
metadata.total_materialized_views += create_query->is_materialized_view;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception & e)
|
catch (Exception & e)
|
||||||
@ -289,10 +291,17 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
|
|||||||
|
|
||||||
size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count;
|
size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count;
|
||||||
size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries;
|
size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries;
|
||||||
|
size_t materialized_views_in_database = metadata.total_materialized_views - prev_total_materialized_views;
|
||||||
size_t tables_in_database = objects_in_database - dictionaries_in_database;
|
size_t tables_in_database = objects_in_database - dictionaries_in_database;
|
||||||
|
|
||||||
LOG_INFO(log, "Metadata processed, database {} has {} tables and {} dictionaries in total.",
|
LOG_INFO(log, "Metadata processed, database {} has {} tables, {} dictionaries and {} materialized views in total.",
|
||||||
TSA_SUPPRESS_WARNING_FOR_READ(database_name), tables_in_database, dictionaries_in_database);
|
TSA_SUPPRESS_WARNING_FOR_READ(database_name), tables_in_database, dictionaries_in_database, materialized_views_in_database);
|
||||||
|
|
||||||
|
// if (materialized_views_in_database)
|
||||||
|
// {
|
||||||
|
|
||||||
|
// }
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void DatabaseOrdinary::loadTableFromMetadata(
|
void DatabaseOrdinary::loadTableFromMetadata(
|
||||||
@ -317,6 +326,8 @@ void DatabaseOrdinary::loadTableFromMetadata(
|
|||||||
mode);
|
mode);
|
||||||
|
|
||||||
attachTable(local_context, table_name, table, getTableDataPath(query));
|
attachTable(local_context, table_name, table, getTableDataPath(query));
|
||||||
|
|
||||||
|
table->pushDependencies();
|
||||||
}
|
}
|
||||||
catch (Exception & e)
|
catch (Exception & e)
|
||||||
{
|
{
|
||||||
|
@ -434,6 +434,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
|
|||||||
for (auto metric : getAttachedCountersForStorage(table))
|
for (auto metric : getAttachedCountersForStorage(table))
|
||||||
CurrentMetrics::add(metric);
|
CurrentMetrics::add(metric);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// if (DatabaseCatalog::iiMa
|
||||||
}
|
}
|
||||||
|
|
||||||
void DatabaseWithOwnTablesBase::shutdown()
|
void DatabaseWithOwnTablesBase::shutdown()
|
||||||
|
@ -44,6 +44,7 @@ struct ParsedTablesMetadata
|
|||||||
|
|
||||||
/// For logging
|
/// For logging
|
||||||
size_t total_dictionaries = 0;
|
size_t total_dictionaries = 0;
|
||||||
|
size_t total_materialized_views = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Loads tables (and dictionaries) from specified databases
|
/// Loads tables (and dictionaries) from specified databases
|
||||||
|
@ -237,6 +237,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
|||||||
null_modifier.emplace(true);
|
null_modifier.emplace(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool is_comment = false;
|
||||||
/// Collate is also allowed after NULL/NOT NULL
|
/// Collate is also allowed after NULL/NOT NULL
|
||||||
if (!collation_expression && s_collate.ignore(pos, expected)
|
if (!collation_expression && s_collate.ignore(pos, expected)
|
||||||
&& !collation_parser.parse(pos, collation_expression, expected))
|
&& !collation_parser.parse(pos, collation_expression, expected))
|
||||||
@ -254,7 +255,9 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
|||||||
else if (s_ephemeral.ignore(pos, expected))
|
else if (s_ephemeral.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
default_specifier = s_ephemeral.getName();
|
default_specifier = s_ephemeral.getName();
|
||||||
if (!expr_parser.parse(pos, default_expression, expected) && type)
|
if (s_comment.ignore(pos, expected))
|
||||||
|
is_comment = true;
|
||||||
|
if ((is_comment || !expr_parser.parse(pos, default_expression, expected)) && type)
|
||||||
{
|
{
|
||||||
ephemeral_default = true;
|
ephemeral_default = true;
|
||||||
|
|
||||||
@ -289,19 +292,22 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
|||||||
if (require_type && !type && !default_expression)
|
if (require_type && !type && !default_expression)
|
||||||
return false; /// reject column name without type
|
return false; /// reject column name without type
|
||||||
|
|
||||||
if ((type || default_expression) && allow_null_modifiers && !null_modifier.has_value())
|
if (!is_comment)
|
||||||
{
|
{
|
||||||
if (s_not.ignore(pos, expected))
|
if ((type || default_expression) && allow_null_modifiers && !null_modifier.has_value())
|
||||||
{
|
{
|
||||||
if (!s_null.ignore(pos, expected))
|
if (s_not.ignore(pos, expected))
|
||||||
return false;
|
{
|
||||||
null_modifier.emplace(false);
|
if (!s_null.ignore(pos, expected))
|
||||||
|
return false;
|
||||||
|
null_modifier.emplace(false);
|
||||||
|
}
|
||||||
|
else if (s_null.ignore(pos, expected))
|
||||||
|
null_modifier.emplace(true);
|
||||||
}
|
}
|
||||||
else if (s_null.ignore(pos, expected))
|
|
||||||
null_modifier.emplace(true);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (s_comment.ignore(pos, expected))
|
if (is_comment || s_comment.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
/// should be followed by a string literal
|
/// should be followed by a string literal
|
||||||
if (!string_literal_parser.parse(pos, comment_expression, expected))
|
if (!string_literal_parser.parse(pos, comment_expression, expected))
|
||||||
|
@ -1,5 +1,4 @@
|
|||||||
#include <Server/KeeperTCPHandler.h>
|
#include <Server/KeeperTCPHandler.h>
|
||||||
#include "Common/ZooKeeper/ZooKeeperConstants.h"
|
|
||||||
|
|
||||||
#if USE_NURAFT
|
#if USE_NURAFT
|
||||||
|
|
||||||
@ -19,6 +18,8 @@
|
|||||||
# include <Common/NetException.h>
|
# include <Common/NetException.h>
|
||||||
# include <Common/PipeFDs.h>
|
# include <Common/PipeFDs.h>
|
||||||
# include <Common/Stopwatch.h>
|
# include <Common/Stopwatch.h>
|
||||||
|
# include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||||
|
# include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||||
# include <Common/ZooKeeper/ZooKeeperIO.h>
|
# include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||||
# include <Common/logger_useful.h>
|
# include <Common/logger_useful.h>
|
||||||
# include <Common/setThreadName.h>
|
# include <Common/setThreadName.h>
|
||||||
@ -63,6 +64,7 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
|
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
|
||||||
extern const int TIMEOUT_EXCEEDED;
|
extern const int TIMEOUT_EXCEEDED;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
struct PollResult
|
struct PollResult
|
||||||
@ -637,7 +639,23 @@ std::pair<Coordination::OpNum, Coordination::XID> KeeperTCPHandler::receiveReque
|
|||||||
|
|
||||||
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
|
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
|
||||||
request->xid = xid;
|
request->xid = xid;
|
||||||
request->readImpl(read_buffer);
|
|
||||||
|
auto request_validator = [&](const Coordination::ZooKeeperRequest & current_request)
|
||||||
|
{
|
||||||
|
if (!keeper_dispatcher->getKeeperContext()->isOperationSupported(current_request.getOpNum()))
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported operation: {}", current_request.getOpNum());
|
||||||
|
};
|
||||||
|
|
||||||
|
if (auto * multi_request = dynamic_cast<Coordination::ZooKeeperMultiRequest *>(request.get()))
|
||||||
|
{
|
||||||
|
multi_request->readImpl(read_buffer, request_validator);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
request->readImpl(read_buffer);
|
||||||
|
request_validator(*request);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
if (!keeper_dispatcher->putRequest(request, session_id, use_xid_64))
|
if (!keeper_dispatcher->putRequest(request, session_id, use_xid_64))
|
||||||
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id);
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id);
|
||||||
|
@ -572,6 +572,11 @@ public:
|
|||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName());
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
virtual void pushDependencies()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/** If the table have to do some complicated work on startup,
|
/** If the table have to do some complicated work on startup,
|
||||||
* that must be postponed after creation of table object
|
* that must be postponed after creation of table object
|
||||||
* (like launching some background threads),
|
* (like launching some background threads),
|
||||||
|
@ -187,6 +187,7 @@ StorageKafka::StorageKafka(
|
|||||||
, thread_per_consumer((*kafka_settings)[KafkaSetting::kafka_thread_per_consumer].value)
|
, thread_per_consumer((*kafka_settings)[KafkaSetting::kafka_thread_per_consumer].value)
|
||||||
, collection_name(collection_name_)
|
, collection_name(collection_name_)
|
||||||
{
|
{
|
||||||
|
LOG_TRACE(log, "Top of StorageKafka ctor.");
|
||||||
kafka_settings->sanityCheck();
|
kafka_settings->sanityCheck();
|
||||||
|
|
||||||
if ((*kafka_settings)[KafkaSetting::kafka_handle_error_mode] == StreamingHandleErrorMode::STREAM)
|
if ((*kafka_settings)[KafkaSetting::kafka_handle_error_mode] == StreamingHandleErrorMode::STREAM)
|
||||||
@ -524,6 +525,7 @@ size_t StorageKafka::getPollTimeoutMillisecond() const
|
|||||||
|
|
||||||
void StorageKafka::threadFunc(size_t idx)
|
void StorageKafka::threadFunc(size_t idx)
|
||||||
{
|
{
|
||||||
|
LOG_DEBUG(log, "Top of StorageKafka::threadFunc");
|
||||||
assert(idx < tasks.size());
|
assert(idx < tasks.size());
|
||||||
auto task = tasks[idx];
|
auto task = tasks[idx];
|
||||||
std::string exception_str;
|
std::string exception_str;
|
||||||
@ -533,15 +535,18 @@ void StorageKafka::threadFunc(size_t idx)
|
|||||||
auto table_id = getStorageID();
|
auto table_id = getStorageID();
|
||||||
// Check if at least one direct dependency is attached
|
// Check if at least one direct dependency is attached
|
||||||
size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size();
|
size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size();
|
||||||
|
LOG_DEBUG(log, "StorageKafka::threadFunc - before if");
|
||||||
if (num_views)
|
if (num_views)
|
||||||
{
|
{
|
||||||
auto start_time = std::chrono::steady_clock::now();
|
auto start_time = std::chrono::steady_clock::now();
|
||||||
|
|
||||||
mv_attached.store(true);
|
mv_attached.store(true);
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "StorageKafka::threadFunc - before while");
|
||||||
// Keep streaming as long as there are attached views and streaming is not cancelled
|
// Keep streaming as long as there are attached views and streaming is not cancelled
|
||||||
while (!task->stream_cancelled)
|
while (!task->stream_cancelled)
|
||||||
{
|
{
|
||||||
|
LOG_DEBUG(log, "StorageKafka::threadFunc - before StorageKafkaUtils::checkDependencies");
|
||||||
if (!StorageKafkaUtils::checkDependencies(table_id, getContext()))
|
if (!StorageKafkaUtils::checkDependencies(table_id, getContext()))
|
||||||
break;
|
break;
|
||||||
|
|
||||||
@ -564,6 +569,9 @@ void StorageKafka::threadFunc(size_t idx)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else
|
||||||
|
LOG_DEBUG(log, "No attached views");
|
||||||
|
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
|
@ -425,15 +425,19 @@ bool checkDependencies(const StorageID & table_id, const ContextPtr& context)
|
|||||||
// Check the dependencies are ready?
|
// Check the dependencies are ready?
|
||||||
for (const auto & view_id : view_ids)
|
for (const auto & view_id : view_ids)
|
||||||
{
|
{
|
||||||
|
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Top of for");
|
||||||
|
|
||||||
auto view = DatabaseCatalog::instance().tryGetTable(view_id, context);
|
auto view = DatabaseCatalog::instance().tryGetTable(view_id, context);
|
||||||
if (!view)
|
if (!view)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Target table");
|
||||||
// If it materialized view, check it's target table
|
// If it materialized view, check it's target table
|
||||||
auto * materialized_view = dynamic_cast<StorageMaterializedView *>(view.get());
|
auto * materialized_view = dynamic_cast<StorageMaterializedView *>(view.get());
|
||||||
if (materialized_view && !materialized_view->tryGetTargetTable())
|
if (materialized_view && !materialized_view->tryGetTargetTable())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Transitive dependencies");
|
||||||
// Check all its dependencies
|
// Check all its dependencies
|
||||||
if (!checkDependencies(view_id, context))
|
if (!checkDependencies(view_id, context))
|
||||||
return false;
|
return false;
|
||||||
|
@ -29,6 +29,7 @@
|
|||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/checkStackSize.h>
|
#include <Common/checkStackSize.h>
|
||||||
|
#include <Common/randomSeed.h>
|
||||||
#include <Core/ServerSettings.h>
|
#include <Core/ServerSettings.h>
|
||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
@ -40,6 +41,8 @@
|
|||||||
|
|
||||||
#include <Backups/BackupEntriesCollector.h>
|
#include <Backups/BackupEntriesCollector.h>
|
||||||
|
|
||||||
|
#include <Common/logger_useful.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
namespace Setting
|
namespace Setting
|
||||||
@ -51,6 +54,7 @@ namespace Setting
|
|||||||
namespace ServerSetting
|
namespace ServerSetting
|
||||||
{
|
{
|
||||||
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
|
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
|
||||||
|
extern const ServerSettingsUInt64 startup_mv_delay_ms;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace RefreshSetting
|
namespace RefreshSetting
|
||||||
@ -745,12 +749,34 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
|||||||
refresher->rename(new_table_id, getTargetTableId());
|
refresher->rename(new_table_id, getTargetTableId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void StorageMaterializedView::pushDependencies()
|
||||||
|
{
|
||||||
|
// assert(!dependencies_are_tracked);
|
||||||
|
if (!dependencies_are_tracked)
|
||||||
|
{
|
||||||
|
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||||
|
const auto & select_query = metadata_snapshot->getSelectQuery();
|
||||||
|
if (!select_query.select_table_id.empty())
|
||||||
|
DatabaseCatalog::instance().addViewDependency(select_query.select_table_id, getStorageID());
|
||||||
|
dependencies_are_tracked = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void StorageMaterializedView::startup()
|
void StorageMaterializedView::startup()
|
||||||
{
|
{
|
||||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
if (const auto configured_delay_ms = getContext()->getServerSettings()[ServerSetting::startup_mv_delay_ms]; configured_delay_ms)
|
||||||
const auto & select_query = metadata_snapshot->getSelectQuery();
|
{
|
||||||
if (!select_query.select_table_id.empty())
|
pcg64_fast gen{randomSeed()};
|
||||||
DatabaseCatalog::instance().addViewDependency(select_query.select_table_id, getStorageID());
|
const auto delay_ms = std::uniform_int_distribution<>(0, 1)(gen) ? configured_delay_ms : 0UL;
|
||||||
|
if (delay_ms)
|
||||||
|
{
|
||||||
|
LOG_DEBUG(&Poco::Logger::get("StorageMaterializedView"), "sleeping in startup of {}", getStorageID().table_name);
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(delay_ms));
|
||||||
|
LOG_DEBUG(&Poco::Logger::get("StorageMaterializedView"), "woken up in startup of {}", getStorageID().table_name);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pushDependencies();
|
||||||
|
|
||||||
if (refresher)
|
if (refresher)
|
||||||
refresher->startup();
|
refresher->startup();
|
||||||
|
@ -67,6 +67,7 @@ public:
|
|||||||
|
|
||||||
void renameInMemory(const StorageID & new_table_id) override;
|
void renameInMemory(const StorageID & new_table_id) override;
|
||||||
|
|
||||||
|
void pushDependencies() override;
|
||||||
void startup() override;
|
void startup() override;
|
||||||
void shutdown(bool is_drop) override;
|
void shutdown(bool is_drop) override;
|
||||||
|
|
||||||
@ -116,6 +117,8 @@ private:
|
|||||||
/// have UUID, and we do inner table lookup by name instead.
|
/// have UUID, and we do inner table lookup by name instead.
|
||||||
bool fixed_uuid = true;
|
bool fixed_uuid = true;
|
||||||
|
|
||||||
|
bool dependencies_are_tracked = false;
|
||||||
|
|
||||||
friend class RefreshTask;
|
friend class RefreshTask;
|
||||||
|
|
||||||
void checkStatementCanBeForwarded() const;
|
void checkStatementCanBeForwarded() const;
|
||||||
|
@ -7,6 +7,7 @@ import os.path as p
|
|||||||
import platform
|
import platform
|
||||||
import pprint
|
import pprint
|
||||||
import pwd
|
import pwd
|
||||||
|
import random
|
||||||
import re
|
import re
|
||||||
import shlex
|
import shlex
|
||||||
import shutil
|
import shutil
|
||||||
@ -1650,6 +1651,8 @@ class ClickHouseCluster:
|
|||||||
minio_certs_dir=None,
|
minio_certs_dir=None,
|
||||||
minio_data_dir=None,
|
minio_data_dir=None,
|
||||||
use_keeper=True,
|
use_keeper=True,
|
||||||
|
keeper_randomize_feature_flags=True,
|
||||||
|
keeper_required_feature_flags=[],
|
||||||
main_config_name="config.xml",
|
main_config_name="config.xml",
|
||||||
users_config_name="users.xml",
|
users_config_name="users.xml",
|
||||||
copy_common_configs=True,
|
copy_common_configs=True,
|
||||||
@ -1682,6 +1685,8 @@ class ClickHouseCluster:
|
|||||||
if not env_variables:
|
if not env_variables:
|
||||||
env_variables = {}
|
env_variables = {}
|
||||||
self.use_keeper = use_keeper
|
self.use_keeper = use_keeper
|
||||||
|
self.keeper_randomize_feature_flags = keeper_randomize_feature_flags
|
||||||
|
self.keeper_required_feature_flags = keeper_required_feature_flags
|
||||||
|
|
||||||
# Code coverage files will be placed in database directory
|
# Code coverage files will be placed in database directory
|
||||||
# (affect only WITH_COVERAGE=1 build)
|
# (affect only WITH_COVERAGE=1 build)
|
||||||
@ -2828,15 +2833,51 @@ class ClickHouseCluster:
|
|||||||
|
|
||||||
if self.use_keeper: # TODO: remove hardcoded paths from here
|
if self.use_keeper: # TODO: remove hardcoded paths from here
|
||||||
for i in range(1, 4):
|
for i in range(1, 4):
|
||||||
|
current_keeper_config_dir = os.path.join(
|
||||||
|
f"{self.keeper_instance_dir_prefix}{i}", "config"
|
||||||
|
)
|
||||||
shutil.copy(
|
shutil.copy(
|
||||||
os.path.join(
|
os.path.join(
|
||||||
self.keeper_config_dir, f"keeper_config{i}.xml"
|
self.keeper_config_dir, f"keeper_config{i}.xml"
|
||||||
),
|
),
|
||||||
os.path.join(
|
current_keeper_config_dir,
|
||||||
self.keeper_instance_dir_prefix + f"{i}", "config"
|
|
||||||
),
|
|
||||||
)
|
)
|
||||||
|
|
||||||
|
extra_configs_dir = os.path.join(
|
||||||
|
current_keeper_config_dir, f"keeper_config{i}.d"
|
||||||
|
)
|
||||||
|
os.mkdir(extra_configs_dir)
|
||||||
|
feature_flags_config = os.path.join(
|
||||||
|
extra_configs_dir, "feature_flags.yaml"
|
||||||
|
)
|
||||||
|
|
||||||
|
indentation = 4 * " "
|
||||||
|
|
||||||
|
def get_feature_flag_value(feature_flag):
|
||||||
|
if not self.keeper_randomize_feature_flags:
|
||||||
|
return 1
|
||||||
|
|
||||||
|
if feature_flag in self.keeper_required_feature_flags:
|
||||||
|
return 1
|
||||||
|
|
||||||
|
return random.randint(0, 1)
|
||||||
|
|
||||||
|
with open(feature_flags_config, "w") as ff_config:
|
||||||
|
ff_config.write("keeper_server:\n")
|
||||||
|
ff_config.write(f"{indentation}feature_flags:\n")
|
||||||
|
indentation *= 2
|
||||||
|
|
||||||
|
for feature_flag in [
|
||||||
|
"filtered_list",
|
||||||
|
"multi_read",
|
||||||
|
"check_not_exists",
|
||||||
|
"create_if_not_exists",
|
||||||
|
"remove_recursive",
|
||||||
|
]:
|
||||||
|
ff_config.write(
|
||||||
|
f"{indentation}{feature_flag}: {get_feature_flag_value(feature_flag)}\n"
|
||||||
|
)
|
||||||
|
|
||||||
run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env)
|
run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env)
|
||||||
self.up_called = True
|
self.up_called = True
|
||||||
|
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
<clickhouse>
|
<clickhouse>
|
||||||
<async_load_databases>true</async_load_databases>
|
<async_load_databases>true</async_load_databases>
|
||||||
|
<startup_mv_delay_ms>10000</startup_mv_delay_ms>
|
||||||
</clickhouse>
|
</clickhouse>
|
||||||
|
@ -242,3 +242,80 @@ def test_async_load_system_database(started_cluster):
|
|||||||
for i in range(id - 1):
|
for i in range(id - 1):
|
||||||
node2.query(f"drop table if exists system.text_log_{i + 1}_test")
|
node2.query(f"drop table if exists system.text_log_{i + 1}_test")
|
||||||
node2.query(f"drop table if exists system.query_log_{i + 1}_test")
|
node2.query(f"drop table if exists system.query_log_{i + 1}_test")
|
||||||
|
|
||||||
|
|
||||||
|
def test_materialzed_views(started_cluster):
|
||||||
|
query = node1.query
|
||||||
|
query("create database test_mv")
|
||||||
|
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.z (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
|
||||||
|
query("create materialized view t_to_z to test_mv.z as select Id from test_mv.t")
|
||||||
|
|
||||||
|
node1.restart_clickhouse()
|
||||||
|
query("insert into test_mv.t values(42)")
|
||||||
|
assert query("select * from test_mv.a Format CSV") == "42\n"
|
||||||
|
assert query("select * from test_mv.z Format CSV") == "42\n"
|
||||||
|
|
||||||
|
query("drop view t_to_a")
|
||||||
|
query("drop view t_to_z")
|
||||||
|
query("drop table test_mv.t")
|
||||||
|
query("drop table test_mv.a")
|
||||||
|
query("drop table test_mv.z")
|
||||||
|
query("drop database test_mv")
|
||||||
|
|
||||||
|
|
||||||
|
def test_materialzed_views_cascaded(started_cluster):
|
||||||
|
query = node1.query
|
||||||
|
query("create database test_mv")
|
||||||
|
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.z (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
|
||||||
|
query("create materialized view a_to_z to test_mv.z as select Id from test_mv.a")
|
||||||
|
|
||||||
|
node1.restart_clickhouse()
|
||||||
|
query("insert into test_mv.t values(42)")
|
||||||
|
assert query("select * from test_mv.a Format CSV") == "42\n"
|
||||||
|
assert query("select * from test_mv.z Format CSV") == "42\n"
|
||||||
|
|
||||||
|
query("drop view t_to_a")
|
||||||
|
query("drop view a_to_z")
|
||||||
|
query("drop table test_mv.t")
|
||||||
|
query("drop table test_mv.a")
|
||||||
|
query("drop table test_mv.z")
|
||||||
|
query("drop database test_mv")
|
||||||
|
|
||||||
|
|
||||||
|
def test_materialzed_views_cascaded_multiple(started_cluster):
|
||||||
|
query = node1.query
|
||||||
|
query("create database test_mv")
|
||||||
|
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
|
||||||
|
query("create table test_mv.x (IdText String) engine=MergeTree order by IdText")
|
||||||
|
query(
|
||||||
|
"create table test_mv.z (Id UInt64, IdTextLength UInt64) engine=MergeTree order by Id"
|
||||||
|
)
|
||||||
|
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
|
||||||
|
query(
|
||||||
|
"create materialized view t_to_x to test_mv.x as select toString(Id) as IdText from test_mv.t"
|
||||||
|
)
|
||||||
|
query(
|
||||||
|
"create materialized view ax_to_z to test_mv.z as select Id, (select max(length(IdText)) from test_mv.x) as IdTextLength from test_mv.a"
|
||||||
|
)
|
||||||
|
|
||||||
|
node1.restart_clickhouse()
|
||||||
|
query("insert into test_mv.t values(42)")
|
||||||
|
assert query("select * from test_mv.a Format CSV") == "42\n"
|
||||||
|
assert query("select * from test_mv.x Format CSV") == '"42"\n'
|
||||||
|
assert query("select * from test_mv.z Format CSV") == "42,2\n"
|
||||||
|
|
||||||
|
query("drop view t_to_a")
|
||||||
|
query("drop view t_to_x")
|
||||||
|
query("drop view ax_to_z")
|
||||||
|
query("drop table test_mv.t")
|
||||||
|
query("drop table test_mv.a")
|
||||||
|
query("drop table test_mv.x")
|
||||||
|
query("drop table test_mv.z")
|
||||||
|
query("drop database test_mv")
|
||||||
|
@ -13,6 +13,7 @@ node = cluster.add_instance(
|
|||||||
main_configs=["configs/enable_keeper_map.xml"],
|
main_configs=["configs/enable_keeper_map.xml"],
|
||||||
user_configs=["configs/keeper_retries.xml"],
|
user_configs=["configs/keeper_retries.xml"],
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["multi_read"],
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -20,6 +20,7 @@ node1 = cluster.add_instance(
|
|||||||
main_configs=["configs/config.xml"],
|
main_configs=["configs/config.xml"],
|
||||||
user_configs=["configs/users.xml"],
|
user_configs=["configs/users.xml"],
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["multi_read", "create_if_not_exists"],
|
||||||
macros={"shard": "shard1", "replica": "1"},
|
macros={"shard": "shard1", "replica": "1"},
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
)
|
)
|
||||||
@ -28,6 +29,7 @@ node2 = cluster.add_instance(
|
|||||||
main_configs=["configs/config.xml"],
|
main_configs=["configs/config.xml"],
|
||||||
user_configs=["configs/users.xml"],
|
user_configs=["configs/users.xml"],
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["multi_read", "create_if_not_exists"],
|
||||||
macros={"shard": "shard1", "replica": "2"},
|
macros={"shard": "shard1", "replica": "2"},
|
||||||
)
|
)
|
||||||
nodes = [node1, node2]
|
nodes = [node1, node2]
|
||||||
|
@ -59,6 +59,9 @@ instance = cluster.add_instance(
|
|||||||
user_configs=["configs/users.xml"],
|
user_configs=["configs/users.xml"],
|
||||||
with_kafka=True,
|
with_kafka=True,
|
||||||
with_zookeeper=True, # For Replicated Table
|
with_zookeeper=True, # For Replicated Table
|
||||||
|
keeper_required_feature_flags=[
|
||||||
|
"create_if_not_exists"
|
||||||
|
], # new Kafka doesn't work without this feature
|
||||||
macros={
|
macros={
|
||||||
"kafka_broker": "kafka1",
|
"kafka_broker": "kafka1",
|
||||||
"kafka_topic_old": KAFKA_TOPIC_OLD,
|
"kafka_topic_old": KAFKA_TOPIC_OLD,
|
||||||
|
@ -99,6 +99,7 @@ def started_cluster():
|
|||||||
with_minio=True,
|
with_minio=True,
|
||||||
with_azurite=True,
|
with_azurite=True,
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/zookeeper.xml",
|
"configs/zookeeper.xml",
|
||||||
"configs/s3queue_log.xml",
|
"configs/s3queue_log.xml",
|
||||||
@ -110,6 +111,7 @@ def started_cluster():
|
|||||||
user_configs=["configs/users.xml"],
|
user_configs=["configs/users.xml"],
|
||||||
with_minio=True,
|
with_minio=True,
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/s3queue_log.xml",
|
"configs/s3queue_log.xml",
|
||||||
],
|
],
|
||||||
@ -118,6 +120,7 @@ def started_cluster():
|
|||||||
cluster.add_instance(
|
cluster.add_instance(
|
||||||
"old_instance",
|
"old_instance",
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
image="clickhouse/clickhouse-server",
|
image="clickhouse/clickhouse-server",
|
||||||
tag="23.12",
|
tag="23.12",
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
@ -127,6 +130,7 @@ def started_cluster():
|
|||||||
cluster.add_instance(
|
cluster.add_instance(
|
||||||
"node1",
|
"node1",
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/zookeeper.xml",
|
"configs/zookeeper.xml",
|
||||||
@ -137,6 +141,7 @@ def started_cluster():
|
|||||||
cluster.add_instance(
|
cluster.add_instance(
|
||||||
"node2",
|
"node2",
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/zookeeper.xml",
|
"configs/zookeeper.xml",
|
||||||
@ -149,6 +154,7 @@ def started_cluster():
|
|||||||
user_configs=["configs/users.xml"],
|
user_configs=["configs/users.xml"],
|
||||||
with_minio=True,
|
with_minio=True,
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/s3queue_log.xml",
|
"configs/s3queue_log.xml",
|
||||||
"configs/merge_tree.xml",
|
"configs/merge_tree.xml",
|
||||||
@ -158,6 +164,7 @@ def started_cluster():
|
|||||||
cluster.add_instance(
|
cluster.add_instance(
|
||||||
"instance_24.5",
|
"instance_24.5",
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
image="clickhouse/clickhouse-server",
|
image="clickhouse/clickhouse-server",
|
||||||
tag="24.5",
|
tag="24.5",
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
@ -170,6 +177,7 @@ def started_cluster():
|
|||||||
cluster.add_instance(
|
cluster.add_instance(
|
||||||
"node_cloud_mode",
|
"node_cloud_mode",
|
||||||
with_zookeeper=True,
|
with_zookeeper=True,
|
||||||
|
keeper_required_feature_flags=["create_if_not_exists"],
|
||||||
stay_alive=True,
|
stay_alive=True,
|
||||||
main_configs=[
|
main_configs=[
|
||||||
"configs/zookeeper.xml",
|
"configs/zookeeper.xml",
|
||||||
|
@ -96,13 +96,15 @@ SELECT 'First JOIN INNER second JOIN INNER';
|
|||||||
First JOIN INNER second JOIN INNER
|
First JOIN INNER second JOIN INNER
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
@ -115,13 +117,15 @@ SELECT 'First JOIN INNER second JOIN LEFT';
|
|||||||
First JOIN INNER second JOIN LEFT
|
First JOIN INNER second JOIN LEFT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
@ -134,17 +138,19 @@ SELECT 'First JOIN INNER second JOIN RIGHT';
|
|||||||
First JOIN INNER second JOIN RIGHT
|
First JOIN INNER second JOIN RIGHT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -156,17 +162,19 @@ SELECT 'First JOIN INNER second JOIN FULL';
|
|||||||
First JOIN INNER second JOIN FULL
|
First JOIN INNER second JOIN FULL
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -178,13 +186,15 @@ SELECT 'First JOIN LEFT second JOIN INNER';
|
|||||||
First JOIN LEFT second JOIN INNER
|
First JOIN LEFT second JOIN INNER
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
@ -197,14 +207,16 @@ SELECT 'First JOIN LEFT second JOIN LEFT';
|
|||||||
First JOIN LEFT second JOIN LEFT
|
First JOIN LEFT second JOIN LEFT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
Join_1_Value_2 String String String
|
Join_1_Value_2 String String String
|
||||||
@ -219,17 +231,19 @@ SELECT 'First JOIN LEFT second JOIN RIGHT';
|
|||||||
First JOIN LEFT second JOIN RIGHT
|
First JOIN LEFT second JOIN RIGHT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -241,19 +255,21 @@ SELECT 'First JOIN LEFT second JOIN FULL';
|
|||||||
First JOIN LEFT second JOIN FULL
|
First JOIN LEFT second JOIN FULL
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
||||||
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
Join_1_Value_2 String String String
|
Join_1_Value_2 String String String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -266,13 +282,15 @@ SELECT 'First JOIN RIGHT second JOIN INNER';
|
|||||||
First JOIN RIGHT second JOIN INNER
|
First JOIN RIGHT second JOIN INNER
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
@ -285,17 +303,19 @@ SELECT 'First JOIN RIGHT second JOIN LEFT';
|
|||||||
First JOIN RIGHT second JOIN LEFT
|
First JOIN RIGHT second JOIN LEFT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String
|
3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String Join_2_Value_3 String String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String Join_2_Value_3 String String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -307,17 +327,19 @@ SELECT 'First JOIN RIGHT second JOIN RIGHT';
|
|||||||
First JOIN RIGHT second JOIN RIGHT
|
First JOIN RIGHT second JOIN RIGHT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -329,19 +351,21 @@ SELECT 'First JOIN RIGHT second JOIN FULL';
|
|||||||
First JOIN RIGHT second JOIN FULL
|
First JOIN RIGHT second JOIN FULL
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String
|
3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String
|
||||||
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
|
String Join_2_Value_3 String String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String Join_2_Value_3 String String
|
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -354,14 +378,16 @@ SELECT 'First JOIN FULL second JOIN INNER';
|
|||||||
First JOIN FULL second JOIN INNER
|
First JOIN FULL second JOIN INNER
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
@ -374,19 +400,21 @@ SELECT 'First JOIN FULL second JOIN LEFT';
|
|||||||
First JOIN FULL second JOIN LEFT
|
First JOIN FULL second JOIN LEFT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
||||||
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String Join_2_Value_3 String String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
Join_1_Value_2 String String String
|
Join_1_Value_2 String String String
|
||||||
String Join_2_Value_3 String String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -399,18 +427,20 @@ SELECT 'First JOIN FULL second JOIN RIGHT';
|
|||||||
First JOIN FULL second JOIN RIGHT
|
First JOIN FULL second JOIN RIGHT
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
|
||||||
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id);
|
||||||
@ -422,21 +452,23 @@ SELECT 'First JOIN FULL second JOIN FULL';
|
|||||||
First JOIN FULL second JOIN FULL
|
First JOIN FULL second JOIN FULL
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
||||||
|
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
||||||
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String
|
||||||
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String
|
||||||
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String
|
||||||
0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String
|
|
||||||
0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
String String Join_3_Value_4 String
|
||||||
|
String Join_2_Value_3 String String
|
||||||
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String
|
||||||
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String
|
||||||
Join_1_Value_2 String String String
|
Join_1_Value_2 String String String
|
||||||
String Join_2_Value_3 String String
|
|
||||||
String String Join_3_Value_4 String
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
--
|
--
|
||||||
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id);
|
||||||
|
@ -64,12 +64,14 @@ SELECT 'First JOIN {{ first_join_type }} second JOIN {{ second_join_type }}';
|
|||||||
|
|
||||||
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
|
||||||
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
|
|
||||||
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value)
|
||||||
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id);
|
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id)
|
||||||
|
ORDER BY ALL;
|
||||||
|
|
||||||
SELECT '--';
|
SELECT '--';
|
||||||
|
|
||||||
|
11
tests/queries/0_stateless/03250_ephemeral_comment.sql
Normal file
11
tests/queries/0_stateless/03250_ephemeral_comment.sql
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
drop table if exists test;
|
||||||
|
CREATE TABLE test (
|
||||||
|
`start_s` UInt32 EPHEMERAL COMMENT 'start UNIX time' ,
|
||||||
|
`start_us` UInt16 EPHEMERAL COMMENT 'start microseconds',
|
||||||
|
`finish_s` UInt32 EPHEMERAL COMMENT 'finish UNIX time',
|
||||||
|
`finish_us` UInt16 EPHEMERAL COMMENT 'finish microseconds',
|
||||||
|
`captured` DateTime MATERIALIZED fromUnixTimestamp(start_s),
|
||||||
|
`duration` Decimal32(6) MATERIALIZED finish_s - start_s + (finish_us - start_us)/1000000
|
||||||
|
)
|
||||||
|
ENGINE Null;
|
||||||
|
drop table if exists test;
|
Loading…
Reference in New Issue
Block a user