mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
40bc3ec0a7
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit ea2516be366a73a02a82b499ed4a7db1d40037e0
|
||||
Subproject commit 7a2d304c21549427460428c9039009ef4bbfd899
|
@ -324,7 +324,6 @@ try
|
||||
using po::value;
|
||||
using Strings = DB::Strings;
|
||||
|
||||
|
||||
po::options_description desc("Allowed options");
|
||||
desc.add_options()
|
||||
("help", "produce help message")
|
||||
|
@ -265,13 +265,12 @@ using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::DefaultHint, M
|
||||
#endif
|
||||
|
||||
/** Allocator with optimization to place small memory ranges in automatic memory.
|
||||
* TODO alignment
|
||||
*/
|
||||
template <typename Base, size_t N = 64>
|
||||
template <typename Base, size_t N = 64, size_t Alignment = 1>
|
||||
class AllocatorWithStackMemory : private Base
|
||||
{
|
||||
private:
|
||||
char stack_memory[N];
|
||||
alignas(Alignment) char stack_memory[N];
|
||||
|
||||
public:
|
||||
/// Do not use boost::noncopyable to avoid the warning about direct base
|
||||
@ -291,7 +290,7 @@ public:
|
||||
return stack_memory;
|
||||
}
|
||||
|
||||
return Base::alloc(size);
|
||||
return Base::alloc(size, Alignment);
|
||||
}
|
||||
|
||||
void free(void * buf, size_t size)
|
||||
@ -308,10 +307,10 @@ public:
|
||||
|
||||
/// Already was big enough to not fit in stack_memory.
|
||||
if (old_size > N)
|
||||
return Base::realloc(buf, old_size, new_size);
|
||||
return Base::realloc(buf, old_size, new_size, Alignment);
|
||||
|
||||
/// Was in stack memory, but now will not fit there.
|
||||
void * new_buf = Base::alloc(new_size);
|
||||
void * new_buf = Base::alloc(new_size, Alignment);
|
||||
memcpy(new_buf, buf, old_size);
|
||||
return new_buf;
|
||||
}
|
||||
|
@ -636,6 +636,6 @@ using PaddedPODArray = PODArray<T, initial_bytes, TAllocator, 15, 16>;
|
||||
template <typename T, size_t inline_bytes,
|
||||
size_t rounded_bytes = integerRoundUp(inline_bytes, sizeof(T))>
|
||||
using PODArrayWithStackMemory = PODArray<T, rounded_bytes,
|
||||
AllocatorWithStackMemory<Allocator<false>, rounded_bytes>>;
|
||||
AllocatorWithStackMemory<Allocator<false>, rounded_bytes, alignof(T)>>;
|
||||
|
||||
}
|
||||
|
@ -200,6 +200,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.") \
|
||||
\
|
||||
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.") \
|
||||
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.") \
|
||||
\
|
||||
M(SettingUInt64, preferred_block_size_bytes, 1000000, "") \
|
||||
\
|
||||
|
@ -128,7 +128,7 @@ public:
|
||||
virtual Block getTotals();
|
||||
|
||||
/// The same for minimums and maximums.
|
||||
Block getExtremes();
|
||||
virtual Block getExtremes();
|
||||
|
||||
|
||||
/** Set the execution progress bar callback.
|
||||
|
@ -32,7 +32,9 @@ public:
|
||||
|
||||
String getName() const override { return "NullAndDoCopy"; }
|
||||
|
||||
Block getHeader() const override { return input->getHeader(); }
|
||||
Block getHeader() const override { return {}; }
|
||||
Block getTotals() override { return {}; }
|
||||
Block getExtremes() override { return {}; }
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
|
@ -366,7 +366,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
params_row[i] = literal->value;
|
||||
}
|
||||
}
|
||||
else if (auto opt_name = getIdentifierName(arguments->children[0]))
|
||||
else if (auto opt_name = tryGetIdentifierName(arguments->children[0]))
|
||||
{
|
||||
function_name = *opt_name;
|
||||
}
|
||||
|
@ -88,7 +88,7 @@ static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & argum
|
||||
params_row[i] = lit->value;
|
||||
}
|
||||
}
|
||||
else if (auto opt_name = getIdentifierName(arguments->children[0]))
|
||||
else if (auto opt_name = tryGetIdentifierName(arguments->children[0]))
|
||||
{
|
||||
function_name = *opt_name;
|
||||
}
|
||||
|
@ -1183,7 +1183,7 @@ struct UnknownMonotonicity
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct ToIntMonotonicity
|
||||
struct ToNumberMonotonicity
|
||||
{
|
||||
static bool has() { return true; }
|
||||
|
||||
@ -1358,21 +1358,21 @@ struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||
struct NameToUUID { static constexpr auto name = "toUUID"; };
|
||||
|
||||
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>>;
|
||||
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>>;
|
||||
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>>;
|
||||
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>>;
|
||||
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>>;
|
||||
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>>;
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToIntMonotonicity<UInt128>>;
|
||||
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToNumberMonotonicity<UInt8>>;
|
||||
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToNumberMonotonicity<UInt16>>;
|
||||
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToNumberMonotonicity<UInt32>>;
|
||||
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToNumberMonotonicity<UInt64>>;
|
||||
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToNumberMonotonicity<Int8>>;
|
||||
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToNumberMonotonicity<Int16>>;
|
||||
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToNumberMonotonicity<Int32>>;
|
||||
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToNumberMonotonicity<Int64>>;
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, ToNumberMonotonicity<Float32>>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, ToNumberMonotonicity<Float64>>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToNumberMonotonicity<UInt16>>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToNumberMonotonicity<UInt32>>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToNumberMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToNumberMonotonicity<UInt32>>;
|
||||
using FunctionToDecimal32 = FunctionConvert<DataTypeDecimal<Decimal32>, NameToDecimal32, UnknownMonotonicity>;
|
||||
using FunctionToDecimal64 = FunctionConvert<DataTypeDecimal<Decimal64>, NameToDecimal64, UnknownMonotonicity>;
|
||||
using FunctionToDecimal128 = FunctionConvert<DataTypeDecimal<Decimal128>, NameToDecimal128, UnknownMonotonicity>;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <IO/WriteBufferAIO.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <functional>
|
||||
#include <filesystem>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
|
@ -463,7 +463,7 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
||||
|
||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||
{
|
||||
auto opt_arg_name = getIdentifierName(lambda_arg_asts[j]);
|
||||
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
|
||||
if (!opt_arg_name)
|
||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
|
@ -39,7 +39,7 @@ struct ColumnNamesContext
|
||||
std::optional<String> name() const
|
||||
{
|
||||
if (expr)
|
||||
return getIdentifierName(expr->database_and_table_name);
|
||||
return tryGetIdentifierName(expr->database_and_table_name);
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
if (is_table)
|
||||
{
|
||||
/// If this is already an external table, you do not need to add anything. Just remember its presence.
|
||||
if (external_tables.end() != external_tables.find(*getIdentifierName(subquery_or_table_name)))
|
||||
if (external_tables.end() != external_tables.find(getIdentifierName(subquery_or_table_name)))
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -74,7 +74,7 @@ void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
|
||||
if (functionIsInOrGlobalInOperator(func_name))
|
||||
{
|
||||
auto & ast = func_arguments->children.at(1);
|
||||
if (auto opt_name = getIdentifierName(ast))
|
||||
if (auto opt_name = tryGetIdentifierName(ast))
|
||||
if (!aliases.count(*opt_name))
|
||||
setIdentifierSpecial(ast);
|
||||
}
|
||||
|
@ -43,6 +43,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INVALID_JOIN_ON_EXPRESSION;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
NameSet removeDuplicateColumns(NamesAndTypesList & columns)
|
||||
@ -538,6 +539,23 @@ void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
||||
}
|
||||
}
|
||||
|
||||
void checkJoin(const ASTTablesInSelectQueryElement * join)
|
||||
{
|
||||
if (!join->table_join)
|
||||
return;
|
||||
|
||||
const auto & table_join = join->table_join->as<ASTTableJoin &>();
|
||||
|
||||
if (table_join.strictness == ASTTableJoin::Strictness::Any)
|
||||
if (table_join.kind != ASTTableJoin::Kind::Left)
|
||||
throw Exception("Old ANY INNER|RIGHT|FULL JOINs are disabled by default. Their logic would be changed. "
|
||||
"Old logic is many-to-one for all kinds of ANY JOINs. It's equil to apply distinct for right table keys. "
|
||||
"Default bahaviour is reserved for many-to-one LEFT JOIN, one-to-many RIGHT JOIN and one-to-one INNER JOIN. "
|
||||
"It would be equal to apply distinct for keys to right, left and both tables respectively. "
|
||||
"Set any_join_distinct_right_table_keys=1 to enable old bahaviour.",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -578,6 +596,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
|
||||
if (const ASTTablesInSelectQueryElement * node = select_query->join())
|
||||
{
|
||||
if (!settings.any_join_distinct_right_table_keys)
|
||||
checkJoin(node);
|
||||
|
||||
if (settings.enable_optimize_predicate_expression)
|
||||
replaceJoinedTable(node);
|
||||
|
||||
|
@ -253,7 +253,7 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D
|
||||
{
|
||||
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
|
||||
for (const auto & key : keys.children)
|
||||
if (auto opt_column = getIdentifierName(key))
|
||||
if (auto opt_column = tryGetIdentifierName(key))
|
||||
data.join_using_columns.insert(*opt_column);
|
||||
else if (key->as<ASTLiteral>())
|
||||
data.join_using_columns.insert(key->getColumnName());
|
||||
|
@ -565,7 +565,7 @@ void executeQuery(
|
||||
}
|
||||
|
||||
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
|
||||
? *getIdentifierName(ast_query_with_output->format)
|
||||
? getIdentifierName(ast_query_with_output->format)
|
||||
: context.getDefaultFormat();
|
||||
|
||||
if (ast_query_with_output && ast_query_with_output->settings_ast)
|
||||
@ -610,7 +610,7 @@ void executeQuery(
|
||||
}
|
||||
|
||||
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
|
||||
? *getIdentifierName(ast_query_with_output->format)
|
||||
? getIdentifierName(ast_query_with_output->format)
|
||||
: context.getDefaultFormat();
|
||||
|
||||
if (ast_query_with_output && ast_query_with_output->settings_ast)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
@ -8,6 +9,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTIdentifier::clone() const
|
||||
{
|
||||
auto ret = std::make_shared<ASTIdentifier>(*this);
|
||||
@ -92,22 +99,32 @@ ASTPtr createTableIdentifier(const String & database_name, const String & table_
|
||||
return database_and_table;
|
||||
}
|
||||
|
||||
std::optional<String> getIdentifierName(const IAST * const ast)
|
||||
String getIdentifierName(const IAST * ast)
|
||||
{
|
||||
if (ast)
|
||||
if (const auto * node = ast->as<ASTIdentifier>())
|
||||
return node->name;
|
||||
String res;
|
||||
if (tryGetIdentifierNameInto(ast, res))
|
||||
return res;
|
||||
throw Exception(ast ? queryToString(*ast) + " is not an identifier" : "AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
||||
std::optional<String> tryGetIdentifierName(const IAST * ast)
|
||||
{
|
||||
String res;
|
||||
if (tryGetIdentifierNameInto(ast, res))
|
||||
return res;
|
||||
return {};
|
||||
}
|
||||
|
||||
bool getIdentifierName(const ASTPtr & ast, String & name)
|
||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name)
|
||||
{
|
||||
if (ast)
|
||||
{
|
||||
if (const auto * node = ast->as<ASTIdentifier>())
|
||||
{
|
||||
name = node->name;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -70,9 +70,12 @@ private:
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
void setIdentifierSpecial(ASTPtr & ast);
|
||||
|
||||
std::optional<String> getIdentifierName(const IAST * const ast);
|
||||
inline std::optional<String> getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||
bool getIdentifierName(const ASTPtr & ast, String & name);
|
||||
String getIdentifierName(const IAST * ast);
|
||||
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
||||
|
||||
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
||||
inline bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name) { return tryGetIdentifierNameInto(ast.get(), name); }
|
||||
|
||||
}
|
||||
|
@ -177,7 +177,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
||||
{
|
||||
if (!name.empty())
|
||||
name += '.';
|
||||
parts.emplace_back(*getIdentifierName(child));
|
||||
parts.emplace_back(getIdentifierName(child));
|
||||
name += parts.back();
|
||||
}
|
||||
|
||||
@ -225,7 +225,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
* If you do not report that the first option is an error, then the argument will be interpreted as 2014 - 01 - 01 - some number,
|
||||
* and the query silently returns an unexpected result.
|
||||
*/
|
||||
if (*getIdentifierName(identifier) == "toDate"
|
||||
if (getIdentifierName(identifier) == "toDate"
|
||||
&& contents_end - contents_begin == strlen("2014-01-01")
|
||||
&& contents_begin[0] >= '2' && contents_begin[0] <= '3'
|
||||
&& contents_begin[1] >= '0' && contents_begin[1] <= '9'
|
||||
@ -267,7 +267,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
|
||||
auto function_node = std::make_shared<ASTFunction>();
|
||||
getIdentifierName(identifier, function_node->name);
|
||||
tryGetIdentifierNameInto(identifier, function_node->name);
|
||||
|
||||
/// func(DISTINCT ...) is equivalent to funcDistinct(...)
|
||||
if (has_distinct_modifier)
|
||||
@ -1158,7 +1158,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
* and in the query "SELECT x FRO FROM t", the word FRO was considered an alias.
|
||||
*/
|
||||
|
||||
const String name = *getIdentifierName(node);
|
||||
const String name = getIdentifierName(node);
|
||||
|
||||
for (const char ** keyword = restricted_keywords; *keyword != nullptr; ++keyword)
|
||||
if (0 == strcasecmp(name.data(), *keyword))
|
||||
@ -1326,7 +1326,7 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
*/
|
||||
bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
|
||||
if (allow_alias_without_as_keyword)
|
||||
if (auto opt_id = getIdentifierName(node))
|
||||
if (auto opt_id = tryGetIdentifierName(node))
|
||||
if (0 == strcasecmp(opt_id->data(), "FROM"))
|
||||
allow_alias_without_as_keyword_now = false;
|
||||
|
||||
@ -1336,7 +1336,7 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
/// FIXME: try to prettify this cast using `as<>()`
|
||||
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get()))
|
||||
{
|
||||
getIdentifierName(alias_node, ast_with_alias->alias);
|
||||
tryGetIdentifierNameInto(alias_node, ast_with_alias->alias);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -354,7 +354,7 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!p_expression.parse(pos, assignment->expression, expected))
|
||||
return false;
|
||||
|
||||
getIdentifierName(column, assignment->column_name);
|
||||
tryGetIdentifierNameInto(column, assignment->column_name);
|
||||
if (assignment->expression)
|
||||
assignment->children.push_back(assignment->expression);
|
||||
|
||||
|
@ -32,13 +32,13 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!table_parser.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
}
|
||||
else
|
||||
{
|
||||
table = database;
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
}
|
||||
|
||||
if (s_partition.ignore(pos, expected))
|
||||
|
@ -39,7 +39,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
|
||||
auto func = std::make_shared<ASTFunction>();
|
||||
getIdentifierName(name, func->name);
|
||||
tryGetIdentifierNameInto(name, func->name);
|
||||
func->arguments = columns;
|
||||
func->children.push_back(columns);
|
||||
node = func;
|
||||
@ -74,7 +74,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
|
||||
if (non_parametric.parse(pos, ident, expected))
|
||||
{
|
||||
auto func = std::make_shared<ASTFunction>();
|
||||
getIdentifierName(ident, func->name);
|
||||
tryGetIdentifierNameInto(ident, func->name);
|
||||
node = func;
|
||||
return true;
|
||||
}
|
||||
@ -384,8 +384,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -542,18 +542,18 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
query->temporary = is_temporary;
|
||||
query->replace_view = replace_view;
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
query->cluster = cluster_str;
|
||||
|
||||
getIdentifierName(to_database, query->to_database);
|
||||
getIdentifierName(to_table, query->to_table);
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
tryGetIdentifierNameInto(to_table, query->to_table);
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
query->set(query->storage, storage);
|
||||
|
||||
getIdentifierName(as_database, query->as_database);
|
||||
getIdentifierName(as_table, query->as_table);
|
||||
tryGetIdentifierNameInto(as_database, query->as_database);
|
||||
tryGetIdentifierNameInto(as_table, query->as_table);
|
||||
query->set(query->select, select);
|
||||
|
||||
return true;
|
||||
|
@ -73,7 +73,7 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expect
|
||||
&& type_parser.parse(pos, type, expected))
|
||||
{
|
||||
auto name_type_pair = std::make_shared<ASTNameTypePair>();
|
||||
getIdentifierName(name, name_type_pair->name);
|
||||
tryGetIdentifierNameInto(name, name_type_pair->name);
|
||||
name_type_pair->type = type;
|
||||
name_type_pair->children.push_back(type);
|
||||
node = name_type_pair;
|
||||
@ -189,7 +189,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
|
||||
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
||||
node = column_declaration;
|
||||
getIdentifierName(name, column_declaration->name);
|
||||
tryGetIdentifierNameInto(name, column_declaration->name);
|
||||
|
||||
if (type)
|
||||
{
|
||||
|
@ -116,8 +116,8 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
|
||||
query->if_exists = if_exists;
|
||||
query->temporary = temporary;
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
|
||||
query->cluster = cluster_str;
|
||||
|
||||
|
@ -147,11 +147,11 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
else
|
||||
{
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
}
|
||||
|
||||
getIdentifierName(format, query->format);
|
||||
tryGetIdentifierNameInto(format, query->format);
|
||||
|
||||
query->columns = columns;
|
||||
query->select = select;
|
||||
|
@ -58,8 +58,8 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
auto query = std::make_shared<ASTOptimizeQuery>();
|
||||
node = query;
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
|
||||
query->cluster = cluster_str;
|
||||
query->partition = partition;
|
||||
|
@ -30,8 +30,8 @@ static bool parseDatabaseAndTable(
|
||||
}
|
||||
|
||||
db_and_table.database.clear();
|
||||
getIdentifierName(database, db_and_table.database);
|
||||
getIdentifierName(table, db_and_table.table);
|
||||
tryGetIdentifierNameInto(database, db_and_table.database);
|
||||
tryGetIdentifierNameInto(table, db_and_table.table);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expec
|
||||
if (!value_p.parse(pos, value, expected))
|
||||
return false;
|
||||
|
||||
getIdentifierName(name, change.name);
|
||||
tryGetIdentifierNameInto(name, change.name);
|
||||
change.value = value->as<ASTLiteral &>().value;
|
||||
|
||||
return true;
|
||||
|
@ -65,7 +65,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
return false;
|
||||
}
|
||||
|
||||
getIdentifierName(database, query->from);
|
||||
tryGetIdentifierNameInto(database, query->from);
|
||||
if (like)
|
||||
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value);
|
||||
|
||||
|
@ -75,8 +75,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
}
|
||||
}
|
||||
|
||||
getIdentifierName(database, query->database);
|
||||
getIdentifierName(table, query->table);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
|
||||
node = query;
|
||||
|
||||
|
@ -21,7 +21,7 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
|
||||
auto query = std::make_shared<ASTUseQuery>();
|
||||
getIdentifierName(database, query->database);
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
node = query;
|
||||
|
||||
return true;
|
||||
|
@ -29,13 +29,13 @@ bool parseDatabaseAndTableName(IParser::Pos & pos, Expected & expected, String &
|
||||
return false;
|
||||
}
|
||||
|
||||
getIdentifierName(database, database_str);
|
||||
getIdentifierName(table, table_str);
|
||||
tryGetIdentifierNameInto(database, database_str);
|
||||
tryGetIdentifierNameInto(table, table_str);
|
||||
}
|
||||
else
|
||||
{
|
||||
database_str = "";
|
||||
getIdentifierName(database, table_str);
|
||||
tryGetIdentifierNameInto(database, table_str);
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -20,7 +20,7 @@ bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, Str
|
||||
result = res->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
else
|
||||
result = *getIdentifierName(res);
|
||||
result = getIdentifierName(res);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -64,7 +64,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type);
|
||||
|
||||
if (command_ast->column)
|
||||
command.after_column = *getIdentifierName(command_ast->column);
|
||||
command.after_column = getIdentifierName(command_ast->column);
|
||||
|
||||
if (ast_col_decl.ttl)
|
||||
command.ttl = ast_col_decl.ttl;
|
||||
@ -80,7 +80,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::DROP_COLUMN;
|
||||
command.column_name = *getIdentifierName(command_ast->column);
|
||||
command.column_name = getIdentifierName(command_ast->column);
|
||||
command.if_exists = command_ast->if_exists;
|
||||
return command;
|
||||
}
|
||||
@ -123,7 +123,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
{
|
||||
AlterCommand command;
|
||||
command.type = COMMENT_COLUMN;
|
||||
command.column_name = *getIdentifierName(command_ast->column);
|
||||
command.column_name = getIdentifierName(command_ast->column);
|
||||
const auto & ast_comment = command_ast->comment->as<ASTLiteral &>();
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
command.if_exists = command_ast->if_exists;
|
||||
|
@ -91,7 +91,7 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
|
||||
|
||||
void ColumnDescription::readText(ReadBuffer & buf)
|
||||
{
|
||||
ParserColumnDeclaration column_parser(true);
|
||||
ParserColumnDeclaration column_parser(/* require type */ true);
|
||||
String column_line;
|
||||
readEscapedStringUntilEOL(column_line, buf);
|
||||
ASTPtr ast = parseQuery(column_parser, column_line, "column parser", 0);
|
||||
|
@ -62,7 +62,7 @@ void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, c
|
||||
|
||||
static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set)
|
||||
{
|
||||
if (auto opt_name = getIdentifierName(ast))
|
||||
if (auto opt_name = tryGetIdentifierName(ast))
|
||||
return (void)set.insert(*opt_name);
|
||||
|
||||
if (ast->as<ASTSubquery>())
|
||||
|
@ -45,13 +45,13 @@ static Names extractColumnNames(const ASTPtr & node)
|
||||
Names res;
|
||||
res.reserve(elements.size());
|
||||
for (const auto & elem : elements)
|
||||
res.push_back(*getIdentifierName(elem));
|
||||
res.push_back(getIdentifierName(elem));
|
||||
|
||||
return res;
|
||||
}
|
||||
else
|
||||
{
|
||||
return { *getIdentifierName(node) };
|
||||
return { getIdentifierName(node) };
|
||||
}
|
||||
}
|
||||
|
||||
@ -502,7 +502,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
{
|
||||
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
|
||||
if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.sign_column))
|
||||
throw Exception(
|
||||
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -514,7 +514,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// If the last element is not index_granularity or replica_name (a literal), then this is the name of the version column.
|
||||
if (!engine_args.empty() && !engine_args.back()->as<ASTLiteral>())
|
||||
{
|
||||
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
|
||||
if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.version_column))
|
||||
throw Exception(
|
||||
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -552,14 +552,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
}
|
||||
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
||||
{
|
||||
if (!getIdentifierName(engine_args.back(), merging_params.version_column))
|
||||
if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.version_column))
|
||||
throw Exception(
|
||||
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
engine_args.pop_back();
|
||||
|
||||
if (!getIdentifierName(engine_args.back(), merging_params.sign_column))
|
||||
if (!tryGetIdentifierNameInto(engine_args.back(), merging_params.sign_column))
|
||||
throw Exception(
|
||||
"Sign column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -616,7 +616,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
|
||||
/// Now only three parameters remain - date (or partitioning expression), primary_key, index_granularity.
|
||||
|
||||
if (!getIdentifierName(engine_args[0], date_column_name))
|
||||
if (!tryGetIdentifierNameInto(engine_args[0], date_column_name))
|
||||
throw Exception(
|
||||
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
@ -65,7 +65,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
PartitionCommand res;
|
||||
res.type = CLEAR_COLUMN;
|
||||
res.partition = command_ast->partition;
|
||||
res.column_name = *getIdentifierName(command_ast->column);
|
||||
res.column_name = getIdentifierName(command_ast->column);
|
||||
return res;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::FREEZE_ALL)
|
||||
|
@ -303,7 +303,7 @@ void registerStorageFile(StorageFactory & factory)
|
||||
{
|
||||
/// Will use FD if engine_args[1] is int literal or identifier with std* name
|
||||
|
||||
if (auto opt_name = getIdentifierName(engine_args[1]))
|
||||
if (auto opt_name = tryGetIdentifierName(engine_args[1]))
|
||||
{
|
||||
if (*opt_name == "stdin")
|
||||
source_fd = STDIN_FILENO;
|
||||
|
@ -90,7 +90,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
"Storage Join requires at least 3 parameters: Join(ANY|ALL, LEFT|INNER, keys...).",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
auto opt_strictness_id = getIdentifierName(engine_args[0]);
|
||||
auto opt_strictness_id = tryGetIdentifierName(engine_args[0]);
|
||||
if (!opt_strictness_id)
|
||||
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
@ -103,7 +103,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
else
|
||||
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto opt_kind_id = getIdentifierName(engine_args[1]);
|
||||
auto opt_kind_id = tryGetIdentifierName(engine_args[1]);
|
||||
if (!opt_kind_id)
|
||||
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
@ -124,7 +124,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
key_names.reserve(engine_args.size() - 2);
|
||||
for (size_t i = 2, size = engine_args.size(); i < size; ++i)
|
||||
{
|
||||
auto opt_key = getIdentifierName(engine_args[i]);
|
||||
auto opt_key = tryGetIdentifierName(engine_args[i]);
|
||||
if (!opt_key)
|
||||
throw Exception("Parameter №" + toString(i + 1) + " of storage Join don't look like column name.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
|
@ -126,6 +126,9 @@ protected:
|
||||
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (column.is_virtual)
|
||||
continue;
|
||||
|
||||
size_t src_index = 0;
|
||||
size_t res_index = 0;
|
||||
|
||||
|
@ -67,7 +67,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!getIdentifierName(args[arg_num], cluster_name))
|
||||
if (!tryGetIdentifierNameInto(args[arg_num], cluster_name))
|
||||
cluster_description = getStringLiteral(*args[arg_num], "Hosts pattern");
|
||||
}
|
||||
++arg_num;
|
||||
|
@ -19,17 +19,17 @@
|
||||
<create_query>DROP TABLE IF EXISTS test_model</create_query>
|
||||
<create_query>CREATE TABLE test_model engine = Memory as select stochasticLinearRegressionState(0.0001)(Age, Income, ParamPrice, Robotness, RefererHash) as state from test.hits</create_query>
|
||||
|
||||
<!-- Check model fit-->
|
||||
<!-- Check model fit -->
|
||||
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15)(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
|
||||
<query>SELECT stochasticLinearRegression(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits</query>
|
||||
|
||||
<!-- Check model fit with Momentum-->
|
||||
<!-- Check model fit with Momentum -->
|
||||
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15, 'Momentum')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
|
||||
|
||||
<!-- Check model fit with Nesterov-->
|
||||
<!-- Check model fit with Nesterov -->
|
||||
<query>WITH (SELECT stochasticLinearRegressionState(0.0001, 0, 15, 'Nesterov')(Age, Income, ParamPrice, Robotness, RefererHash) FROM test.hits) AS model SELECT 1</query>
|
||||
|
||||
<!-- Check model predict-->
|
||||
<!-- Check model predict -->
|
||||
<query>with (SELECT state FROM test_model) as model select evalMLMethod(model, Income, ParamPrice, Robotness, RefererHash) from test.hits</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS test_model</drop_query>
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT a.*, b.* FROM
|
||||
(
|
||||
SELECT number AS k FROM system.numbers LIMIT 10
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ANY INNER JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
|
||||
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
|
||||
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING a,b ORDER BY a,b,c,d ASC;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
|
||||
SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
|
||||
SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS series;
|
||||
|
||||
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS series;
|
||||
|
||||
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT k, x, y FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
|
||||
SELECT k, x FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
|
||||
SELECT k, y FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
|
||||
@ -20,8 +22,11 @@ INSERT INTO t1_00203 VALUES (1, 2, 3, 'aaa'), (2, 3, 4, 'bbb');
|
||||
INSERT INTO t2_00203 VALUES ('ccc', 4, 3, 2), ('ddd', 7, 6, 5);
|
||||
|
||||
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
|
||||
|
||||
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
|
||||
|
||||
SET any_join_distinct_right_table_keys = 0;
|
||||
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 }
|
||||
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 }
|
||||
|
||||
DROP TABLE t1_00203;
|
||||
DROP TABLE t2_00203;
|
||||
|
@ -1,11 +1 @@
|
||||
|
||||
1
|
||||
1
|
||||
|
||||
1
|
||||
|
||||
1
|
||||
|
||||
1
|
||||
1
|
||||
4 1 1
|
||||
|
@ -1,3 +0,0 @@
|
||||
|
||||
1
|
||||
1
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING a, b;
|
||||
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING b, a;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT
|
||||
floor((ReferrerTimestamp - InstallTimestamp) / 86400) AS DaysSinceInstallations
|
||||
FROM
|
||||
|
@ -1 +1,2 @@
|
||||
set any_join_distinct_right_table_keys = 1;
|
||||
select a from (select (1, 2) as a) any inner join (select (1, 2) as a) using a;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
select x, y from (select 1 as x, 2 as y, x, y);
|
||||
select x, y from (select 1 as x, 1 as y, x, y);
|
||||
select x from (select 1 as x, 1 as y, x, y);
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET join_use_nulls = 0;
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT k, a, b
|
||||
FROM
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET join_use_nulls = 0;
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT k, a, b
|
||||
FROM
|
||||
|
@ -14,7 +14,7 @@ INSERT INTO test2_00550 VALUES ('a');
|
||||
CREATE TABLE test3_00550 ( id String, name String ) ENGINE = StripeLog;
|
||||
INSERT INTO test3_00550 VALUES ('a', 'aaa');
|
||||
|
||||
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY INNER JOIN test3_00550 USING (id);
|
||||
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY INNER JOIN test3_00550 USING (id) SETTINGS any_join_distinct_right_table_keys=1;
|
||||
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY LEFT OUTER JOIN test3_00550 USING (id);
|
||||
|
||||
DROP TABLE test1_00550;
|
||||
|
@ -1,2 +1,3 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
SET max_block_size = 10;
|
||||
SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s, s as x from numbers(11)) using (s) ORDER BY s;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
drop table IF EXISTS joinbug;
|
||||
|
||||
CREATE TABLE joinbug (
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT k, a1, b1, a2, b2 FROM (SELECT 0 AS k, 'hello' AS a1, 123 AS b1, a1) ANY FULL OUTER JOIN (SELECT 1 AS k, 'hello' AS a2, 456 AS b2, a2) USING (k) ORDER BY k;
|
||||
SELECT k, a, b FROM (SELECT 0 AS k, 'hello' AS a, 123 AS b, a) ANY FULL OUTER JOIN (SELECT 1 AS k) USING (k) ORDER BY k;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS local_statements;
|
||||
DROP TABLE IF EXISTS statements;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test_00597;
|
||||
DROP TABLE IF EXISTS test_view_00597;
|
||||
|
@ -80,7 +80,7 @@ FROM
|
||||
|
||||
# Check per-thread and per-query ProfileEvents consistency
|
||||
|
||||
$CLICKHOUSE_CLIENT $settings -q "
|
||||
$CLICKHOUSE_CLIENT $settings --any_join_distinct_right_table_keys=1 -q "
|
||||
SELECT PN, PVq, PVt FROM
|
||||
(
|
||||
SELECT PN, sum(PV) AS PVt
|
||||
|
@ -1,3 +1,5 @@
|
||||
set any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT * FROM (SELECT 1 AS id, 2 AS value);
|
||||
SELECT * FROM (SELECT 1 AS id, 2 AS value, 3 AS A) ANY INNER JOIN (SELECT 1 AS id, 4 AS values, 5 AS D) USING id;
|
||||
SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) ANY INNER JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;
|
||||
|
@ -12,6 +12,7 @@ SYSTEM SYNC REPLICA byte_identical_r2;
|
||||
ALTER TABLE byte_identical_r1 ADD COLUMN y DEFAULT rand();
|
||||
OPTIMIZE TABLE byte_identical_r1 PARTITION tuple() FINAL;
|
||||
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
SELECT x, t1.y - t2.y FROM byte_identical_r1 t1 ANY INNER JOIN byte_identical_r2 t2 USING x ORDER BY x;
|
||||
|
||||
DROP TABLE byte_identical_r1;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) join (SELECT 1 as a, 'y' as b) using a;
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a;
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) full join (SELECT 1 as a, 'y' as b) using a;
|
||||
@ -7,3 +9,7 @@ SELECT * FROM (SELECT 1 AS a, 'x' AS b) any join (SELECT 1 as a, 'y' as b) using
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any left join (SELECT 1 as a, 'y' as b) using a;
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a;
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a;
|
||||
|
||||
SET any_join_distinct_right_table_keys = 0;
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 }
|
||||
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 }
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT * FROM (SELECT dummy as a, (toUInt8(0), toUInt8(0)) AS tup FROM system.one)
|
||||
JOIN (SELECT dummy as a, (toUInt8(0), toUInt8(0)) AS tup FROM system.one)
|
||||
USING (a, tup);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t1_00848;
|
||||
DROP TABLE IF EXISTS t2_00848;
|
||||
DROP TABLE IF EXISTS t3_00848;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set any_join_distinct_right_table_keys = 1;
|
||||
|
||||
drop table if exists fooL;
|
||||
drop table if exists fooR;
|
||||
create table fooL (a Int32, v String) engine = Memory;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT * FROM (SELECT 1 AS x) ALL LEFT JOIN (SELECT 1 AS x) USING x;
|
||||
SELECT * FROM (SELECT 1 AS x) ALL LEFT JOIN (SELECT 2 AS x) USING x;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test.filtered_table1;
|
||||
DROP TABLE IF EXISTS test.filtered_table2;
|
||||
DROP TABLE IF EXISTS test.filtered_table3;
|
||||
|
@ -1,3 +1,4 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
SET join_use_nulls = 1;
|
||||
SELECT number FROM system.numbers ANY INNER JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;
|
||||
SELECT number FROM system.numbers ANY LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;
|
||||
SELECT number FROM system.numbers ANY LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;
|
||||
|
@ -0,0 +1,6 @@
|
||||
7777
|
||||
7777
|
||||
7777
|
||||
7777
|
||||
7777.000
|
||||
7777.000
|
@ -0,0 +1,23 @@
|
||||
DROP TABLE IF EXISTS test1;
|
||||
DROP TABLE IF EXISTS test2;
|
||||
DROP TABLE IF EXISTS test3;
|
||||
|
||||
CREATE TABLE test1 (n UInt64) ENGINE = MergeTree ORDER BY n SETTINGS index_granularity = 1;
|
||||
CREATE TABLE test2 (s String) ENGINE = MergeTree ORDER BY s SETTINGS index_granularity = 1;
|
||||
CREATE TABLE test3 (d Decimal(4, 3)) ENGINE = MergeTree ORDER BY d SETTINGS index_granularity = 1;
|
||||
|
||||
INSERT INTO test1 SELECT * FROM numbers(10000);
|
||||
SELECT n FROM test1 WHERE toFloat64(n) = 7777.0 SETTINGS max_rows_to_read = 2;
|
||||
SELECT n FROM test1 WHERE toFloat32(n) = 7777.0 SETTINGS max_rows_to_read = 2;
|
||||
|
||||
INSERT INTO test2 SELECT toString(number) FROM numbers(10000);
|
||||
SELECT s FROM test2 WHERE toFloat64(s) = 7777.0;
|
||||
SELECT s FROM test2 WHERE toFloat32(s) = 7777.0;
|
||||
|
||||
INSERT INTO test3 SELECT toDecimal64(number, 3) FROM numbers(10000);
|
||||
SELECT d FROM test3 WHERE toFloat64(d) = 7777.0 SETTINGS max_rows_to_read = 2;
|
||||
SELECT d FROM test3 WHERE toFloat32(d) = 7777.0 SETTINGS max_rows_to_read = 2;
|
||||
|
||||
DROP TABLE test1;
|
||||
DROP TABLE test2;
|
||||
DROP TABLE test3;
|
@ -0,0 +1,7 @@
|
||||
1.00
|
||||
1.00
|
||||
1.00
|
||||
1.00
|
||||
1.00
|
||||
1.00
|
||||
1.00
|
@ -0,0 +1,8 @@
|
||||
select arrayReduce('median', [toDecimal32OrNull('1', 2)]);
|
||||
select arrayReduce('median', [toDecimal64OrNull('1', 2)]);
|
||||
select arrayReduce('median', [toDecimal128OrZero('1', 2)]);
|
||||
select arrayReduce('sum', [toDecimal128OrNull('1', 2)]);
|
||||
|
||||
select arrayReduce('median', [toDecimal128OrNull('1', 2)]);
|
||||
select arrayReduce('quantile(0.2)', [toDecimal128OrNull('1', 2)]);
|
||||
select arrayReduce('medianExact', [toDecimal128OrNull('1', 2)]);
|
@ -0,0 +1 @@
|
||||
default merge_ab x UInt8 0 0 0 0 0 0 0
|
@ -0,0 +1,13 @@
|
||||
DROP TABLE IF EXISTS merge_a;
|
||||
DROP TABLE IF EXISTS merge_b;
|
||||
DROP TABLE IF EXISTS merge_ab;
|
||||
|
||||
CREATE TABLE merge_a (x UInt8) ENGINE = StripeLog;
|
||||
CREATE TABLE merge_b (x UInt8) ENGINE = StripeLog;
|
||||
CREATE TABLE merge_ab AS merge(currentDatabase(), '^merge_[ab]$');
|
||||
|
||||
SELECT * FROM system.columns WHERE database = currentDatabase() AND table = 'merge_ab';
|
||||
|
||||
DROP TABLE merge_a;
|
||||
DROP TABLE merge_b;
|
||||
DROP TABLE merge_ab;
|
@ -0,0 +1,13 @@
|
||||
CREATE TABLE xx
|
||||
(
|
||||
`date` Date,
|
||||
`id` Int64,
|
||||
`clicks` Int64,
|
||||
`price` Float64,
|
||||
`spend` Float64
|
||||
)
|
||||
ENGINE = SummingMergeTree([price, spend])
|
||||
PARTITION BY toYYYYMM(date)
|
||||
ORDER BY id
|
||||
SAMPLE BY id
|
||||
SETTINGS index_granularity = 8192; -- { serverError 223 }
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT
|
||||
loyalty,
|
||||
count()
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET any_join_distinct_right_table_keys = 1;
|
||||
|
||||
USE test;
|
||||
|
||||
DROP TABLE IF EXISTS join;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set any_join_distinct_right_table_keys = 1;
|
||||
|
||||
SELECT
|
||||
CounterID,
|
||||
hits,
|
||||
|
Loading…
Reference in New Issue
Block a user