mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-19 14:11:58 +00:00
Merge remote-tracking branch 'upstream/master' into fix25
This commit is contained in:
commit
5eca86395a
@ -305,9 +305,9 @@ void MySQLHandler::authenticate(const HandshakeResponse & handshake_response, co
|
||||
LOG_TRACE(log, "Received empty password");
|
||||
}
|
||||
|
||||
if (!password.empty())
|
||||
if (!password.empty() && password.back() == 0)
|
||||
{
|
||||
password.pop_back(); /// terminating null byte
|
||||
password.pop_back();
|
||||
}
|
||||
|
||||
try
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,9 +25,11 @@ struct AsteriskSemantic
|
||||
|
||||
static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
static void setAliases(ASTColumnsMatcher & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
|
||||
static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
static RevertedAliasesPtr getAliases(const ASTColumnsMatcher & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
|
||||
private:
|
||||
static std::shared_ptr<AsteriskSemanticImpl> makeSemantic(const RevertedAliasesPtr & aliases)
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
@ -511,12 +511,14 @@ public:
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
for (auto & [name, info] : infos)
|
||||
{
|
||||
if ((info.was_loading() || load_never_loading) && filter_by_name(name))
|
||||
{
|
||||
cancelLoading(info);
|
||||
info.forced_to_reload = true;
|
||||
startLoading(name, info);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Starts reloading of all the objects.
|
||||
@ -528,20 +530,22 @@ public:
|
||||
/// The function doesn't touch the objects which were never tried to load.
|
||||
void reloadOutdated()
|
||||
{
|
||||
/// Iterate through all the objects and find loaded ones which should be checked if they were modified.
|
||||
std::unordered_map<LoadablePtr, bool> is_modified_map;
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
TimePoint now = std::chrono::system_clock::now();
|
||||
for (const auto & name_and_info : infos)
|
||||
{
|
||||
const auto & info = name_and_info.second;
|
||||
if ((now >= info.next_update_time) && !info.loading() && info.was_loading())
|
||||
if ((now >= info.next_update_time) && !info.loading() && info.loaded())
|
||||
is_modified_map.emplace(info.object, true);
|
||||
}
|
||||
}
|
||||
|
||||
/// The `mutex` should be unlocked while we're calling the function is_object_modified().
|
||||
/// Find out which of the loaded objects were modified.
|
||||
/// We couldn't perform these checks while we were building `is_modified_map` because
|
||||
/// the `mutex` should be unlocked while we're calling the function is_object_modified().
|
||||
for (auto & [object, is_modified_flag] : is_modified_map)
|
||||
{
|
||||
try
|
||||
@ -554,21 +558,38 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
/// Iterate through all the objects again and either start loading or just set `next_update_time`.
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
TimePoint now = std::chrono::system_clock::now();
|
||||
for (auto & [name, info] : infos)
|
||||
if ((now >= info.next_update_time) && !info.loading() && info.was_loading())
|
||||
{
|
||||
if ((now >= info.next_update_time) && !info.loading())
|
||||
{
|
||||
auto it = is_modified_map.find(info.object);
|
||||
if (it == is_modified_map.end())
|
||||
continue; /// Object has been just added, it can be simply omitted from this update of outdated.
|
||||
bool is_modified_flag = it->second;
|
||||
if (info.loaded() && !is_modified_flag)
|
||||
info.next_update_time = calculate_next_update_time(info.object, info.error_count);
|
||||
else
|
||||
if (info.loaded())
|
||||
{
|
||||
auto it = is_modified_map.find(info.object);
|
||||
if (it == is_modified_map.end())
|
||||
continue; /// Object has been just loaded (it wasn't loaded while we were building the map `is_modified_map`), so we don't have to reload it right now.
|
||||
|
||||
bool is_modified_flag = it->second;
|
||||
if (!is_modified_flag)
|
||||
{
|
||||
/// Object wasn't modified so we only have to set `next_update_time`.
|
||||
info.next_update_time = calculate_next_update_time(info.object, info.error_count);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Object was modified and should be reloaded.
|
||||
startLoading(name, info);
|
||||
}
|
||||
else if (info.failed())
|
||||
{
|
||||
/// Object was never loaded successfully and should be reloaded.
|
||||
startLoading(name, info);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
@ -25,6 +26,7 @@
|
||||
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -266,7 +268,7 @@ std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(con
|
||||
continue;
|
||||
}
|
||||
}
|
||||
idx++;
|
||||
++idx;
|
||||
}
|
||||
}
|
||||
return predicate_expressions;
|
||||
@ -412,7 +414,7 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
|
||||
|
||||
for (const auto & projection_column : select_query->select()->children)
|
||||
{
|
||||
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>())
|
||||
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>() || projection_column->as<ASTColumnsMatcher>())
|
||||
{
|
||||
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
|
||||
|
||||
@ -483,8 +485,20 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
||||
throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto block = storage->getSampleBlock();
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
||||
if (const auto * asterisk_pattern = asterisk->as<ASTColumnsMatcher>())
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); ++idx)
|
||||
{
|
||||
auto & col = block.getByPosition(idx);
|
||||
if (asterisk_pattern->isColumnMatching(col.name))
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(col.name));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); ++idx)
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
||||
}
|
||||
}
|
||||
}
|
||||
return projection_columns;
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
@ -41,6 +41,7 @@ namespace ErrorCodes
|
||||
extern const int EMPTY_NESTED_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INVALID_JOIN_ON_EXPRESSION;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
|
||||
}
|
||||
|
||||
NameSet removeDuplicateColumns(NamesAndTypesList & columns)
|
||||
@ -110,6 +111,10 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query
|
||||
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns);
|
||||
TranslateQualifiedNamesVisitor visitor(visitor_data, log.stream());
|
||||
visitor.visit(query);
|
||||
|
||||
/// This may happen after expansion of COLUMNS('regexp').
|
||||
if (select_query.select()->children.empty())
|
||||
throw Exception("Empty list of columns in SELECT query", ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
||||
}
|
||||
|
||||
bool hasArrayJoin(const ASTPtr & ast)
|
||||
|
@ -16,7 +16,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <iostream>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -124,7 +124,6 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
|
||||
if (auto join = select.join())
|
||||
extractJoinUsingColumns(join->table_join, data);
|
||||
|
||||
#if 1 /// TODO: legacy?
|
||||
/// If the WHERE clause or HAVING consists of a single qualified column, the reference must be translated not only in children,
|
||||
/// but also in where_expression and having_expression.
|
||||
if (select.prewhere())
|
||||
@ -133,7 +132,6 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr
|
||||
Visitor(data).visit(select.refWhere());
|
||||
if (select.having())
|
||||
Visitor(data).visit(select.refHaving());
|
||||
#endif
|
||||
}
|
||||
|
||||
static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases)
|
||||
@ -166,7 +164,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
bool has_asterisk = false;
|
||||
for (const auto & child : node.children)
|
||||
{
|
||||
if (child->as<ASTAsterisk>())
|
||||
if (child->as<ASTAsterisk>() || child->as<ASTColumnsMatcher>())
|
||||
{
|
||||
if (tables_with_columns.empty())
|
||||
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -207,6 +205,23 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
first_table = false;
|
||||
}
|
||||
}
|
||||
else if (const auto * asterisk_pattern = child->as<ASTColumnsMatcher>())
|
||||
{
|
||||
bool first_table = true;
|
||||
for (const auto & [table, table_columns] : tables_with_columns)
|
||||
{
|
||||
for (const auto & column_name : table_columns)
|
||||
{
|
||||
if (asterisk_pattern->isColumnMatching(column_name) && (first_table || !data.join_using_columns.count(column_name)))
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
|
||||
}
|
||||
}
|
||||
|
||||
first_table = false;
|
||||
}
|
||||
}
|
||||
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
||||
|
45
dbms/src/Parsers/ASTColumnsMatcher.cpp
Normal file
45
dbms/src/Parsers/ASTColumnsMatcher.cpp
Normal file
@ -0,0 +1,45 @@
|
||||
#include "ASTColumnsMatcher.h"
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTColumnsMatcher::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTColumnsMatcher>(*this);
|
||||
clone->cloneChildren();
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsMatcher::appendColumnName(WriteBuffer & ostr) const { writeString(original_pattern, ostr); }
|
||||
|
||||
void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
WriteBufferFromOwnString pattern_quoted;
|
||||
writeQuotedString(original_pattern, pattern_quoted);
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(" << pattern_quoted.str() << ")";
|
||||
}
|
||||
|
||||
void ASTColumnsMatcher::setPattern(String pattern)
|
||||
{
|
||||
original_pattern = std::move(pattern);
|
||||
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
|
||||
if (!column_matcher->ok())
|
||||
throw DB::Exception("COLUMNS pattern " + original_pattern + " cannot be compiled: " + column_matcher->error(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
|
||||
}
|
||||
|
||||
bool ASTColumnsMatcher::isColumnMatching(const String & column_name) const
|
||||
{
|
||||
return RE2::PartialMatch(column_name, *column_matcher);
|
||||
}
|
||||
|
||||
|
||||
}
|
50
dbms/src/Parsers/ASTColumnsMatcher.h
Normal file
50
dbms/src/Parsers/ASTColumnsMatcher.h
Normal file
@ -0,0 +1,50 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
||||
namespace re2
|
||||
{
|
||||
class RE2;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
}
|
||||
|
||||
struct AsteriskSemantic;
|
||||
struct AsteriskSemanticImpl;
|
||||
|
||||
|
||||
/** SELECT COLUMNS('regexp') is expanded to multiple columns like * (asterisk).
|
||||
*/
|
||||
class ASTColumnsMatcher : public IAST
|
||||
{
|
||||
public:
|
||||
String getID(char) const override { return "ColumnsMatcher"; }
|
||||
ASTPtr clone() const override;
|
||||
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void setPattern(String pattern);
|
||||
bool isColumnMatching(const String & column_name) const;
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<re2::RE2> column_matcher;
|
||||
String original_pattern;
|
||||
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
|
||||
|
||||
friend struct AsteriskSemantic;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -1,7 +1,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
@ -29,6 +29,7 @@
|
||||
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include "ASTColumnsMatcher.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -1168,6 +1169,34 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
|
||||
|
||||
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword columns("COLUMNS");
|
||||
ParserStringLiteral regex;
|
||||
|
||||
if (!columns.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::OpeningRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
ASTPtr regex_node;
|
||||
if (!regex.parse(pos, regex_node, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
auto res = std::make_shared<ASTColumnsMatcher>();
|
||||
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
|
||||
res->children.push_back(regex_node);
|
||||
node = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
|
||||
{
|
||||
if (pos->type == TokenType::Asterisk)
|
||||
@ -1262,6 +1291,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
|| ParserLeftExpression().parse(pos, node, expected)
|
||||
|| ParserRightExpression().parse(pos, node, expected)
|
||||
|| ParserCase().parse(pos, node, expected)
|
||||
|| ParserColumnsMatcher().parse(pos, node, expected) /// before ParserFunction because it can be also parsed as a function.
|
||||
|| ParserFunction().parse(pos, node, expected)
|
||||
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|
||||
|| ParserAsterisk().parse(pos, node, expected)
|
||||
|
@ -56,7 +56,6 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
/// Just *
|
||||
class ParserAsterisk : public IParserBase
|
||||
{
|
||||
@ -65,7 +64,6 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
/** Something like t.* or db.table.*
|
||||
*/
|
||||
class ParserQualifiedAsterisk : public IParserBase
|
||||
@ -75,6 +73,14 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/** COLUMNS('<regular expression>')
|
||||
*/
|
||||
class ParserColumnsMatcher : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "COLUMNS matcher"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/** A function, for example, f(x, y + 1, g(z)).
|
||||
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
|
||||
|
@ -530,7 +530,11 @@ bool StorageMergeTree::merge(
|
||||
}
|
||||
|
||||
if (!selected)
|
||||
{
|
||||
if (out_disable_reason)
|
||||
*out_disable_reason = "Cannot select parts for optimization";
|
||||
return false;
|
||||
}
|
||||
|
||||
merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this);
|
||||
}
|
||||
@ -886,8 +890,16 @@ bool StorageMergeTree::optimize(
|
||||
{
|
||||
if (!merge(true, partition_id, true, deduplicate, &disable_reason))
|
||||
{
|
||||
std::stringstream message;
|
||||
message << "Cannot OPTIMIZE table";
|
||||
if (!disable_reason.empty())
|
||||
message << ": " << disable_reason;
|
||||
else
|
||||
message << " by some reason.";
|
||||
LOG_INFO(log, message.rdbuf());
|
||||
|
||||
if (context.getSettingsRef().optimize_throw_if_noop)
|
||||
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
|
||||
throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -900,8 +912,16 @@ bool StorageMergeTree::optimize(
|
||||
|
||||
if (!merge(true, partition_id, final, deduplicate, &disable_reason))
|
||||
{
|
||||
std::stringstream message;
|
||||
message << "Cannot OPTIMIZE table";
|
||||
if (!disable_reason.empty())
|
||||
message << ": " << disable_reason;
|
||||
else
|
||||
message << " by some reason.";
|
||||
LOG_INFO(log, message.rdbuf());
|
||||
|
||||
if (context.getSettingsRef().optimize_throw_if_noop)
|
||||
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
|
||||
throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -3042,8 +3042,12 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
|
||||
|
||||
if (!selected)
|
||||
{
|
||||
LOG_INFO(log, "Cannot select parts for optimization" + (disable_reason.empty() ? "" : ": " + disable_reason));
|
||||
return handle_noop(disable_reason);
|
||||
std::stringstream message;
|
||||
message << "Cannot select parts for optimization";
|
||||
if (!disable_reason.empty())
|
||||
message << ": " << disable_reason;
|
||||
LOG_INFO(log, message.rdbuf());
|
||||
return handle_noop(message.str());
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeLogEntryData merge_entry;
|
||||
|
@ -1,9 +1,12 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
@ -11,10 +14,7 @@
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <iostream>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -0,0 +1,7 @@
|
||||
FROM php:7.3-cli
|
||||
|
||||
COPY ./client.crt client.crt
|
||||
COPY ./client.key client.key
|
||||
COPY ./test.php test.php
|
||||
|
||||
RUN docker-php-ext-install pdo pdo_mysql
|
@ -0,0 +1,18 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIC+zCCAeOgAwIBAgIJAIhI9ozZJ+TWMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
|
||||
BAMMCWxvY2FsaG9zdDAeFw0xOTA0MjIwNDMyNTJaFw0yMDA0MjEwNDMyNTJaMBQx
|
||||
EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
|
||||
ggEBAK+wVUEdqF2uXvN0MJBgnAHyXi6JTi4p/F6igsrCjSNjJWzHH0vQmK8ujfcF
|
||||
CkifW88i+W5eHctuEtQqNHK+t9x9YiZtXrj6m/XkOXs20mYgENSmbbbHbriTPnZB
|
||||
zZrq6UqMlwIHNNAa+I3NMORQxVRaI0ybXnGVO5elr70xHpk03xL0JWKHpEqYp4db
|
||||
2aBQgF6y3Ww4khxjIYqpUYXWXGFnVIRU7FKVEAM1xyKqvQzXjQ5sVM/wyHknveEF
|
||||
3b/X4ggN+KNl5KOc0cWDh1/XaatJAPaUUPqZcq76tynLbP64Xm3dxHcj+gtRkO67
|
||||
ef6MSg6l63m3XQP6Qb+MIkd06OsCAwEAAaNQME4wHQYDVR0OBBYEFDmODTO8QLDN
|
||||
ykR3x0LIOnjNhrKhMB8GA1UdIwQYMBaAFDmODTO8QLDNykR3x0LIOnjNhrKhMAwG
|
||||
A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAAwaiJc7uqEpnH3aukbftDwX
|
||||
m8GfEnj1HVdgg+9GGNq+9rvUYBF6gdPmjRCX9dO0cclLFx8jc2org0rTSq9WoOhX
|
||||
E6qL4Eqrmc5SE3Y9jZM0h6GRD4oXK014FmtZ3T6ddZU3dQLj3BS2r1XrvmubTvGN
|
||||
ZuTJNY8nx8Hh6H5XINmsEjUF9E5hog+PwCE03xt2adIdYL+gsbxASeNYyeUFpZv5
|
||||
zcXR3VoakBWnAaOVgCHq2qh96QAnL7ZKzFkGf/MdwV10KU3dmb+ICbQUUdf9Gc17
|
||||
aaDCIRws312F433FdXBkGs2UkB7ZZme9dfn6O1QbeTNvex2VLMqYx/CTkfFbOQA=
|
||||
-----END CERTIFICATE-----
|
@ -0,0 +1,28 @@
|
||||
-----BEGIN PRIVATE KEY-----
|
||||
MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCvsFVBHahdrl7z
|
||||
dDCQYJwB8l4uiU4uKfxeooLKwo0jYyVsxx9L0JivLo33BQpIn1vPIvluXh3LbhLU
|
||||
KjRyvrfcfWImbV64+pv15Dl7NtJmIBDUpm22x264kz52Qc2a6ulKjJcCBzTQGviN
|
||||
zTDkUMVUWiNMm15xlTuXpa+9MR6ZNN8S9CVih6RKmKeHW9mgUIBest1sOJIcYyGK
|
||||
qVGF1lxhZ1SEVOxSlRADNcciqr0M140ObFTP8Mh5J73hBd2/1+IIDfijZeSjnNHF
|
||||
g4df12mrSQD2lFD6mXKu+rcpy2z+uF5t3cR3I/oLUZDuu3n+jEoOpet5t10D+kG/
|
||||
jCJHdOjrAgMBAAECggEARF66zrxb6RkSmmt8+rKeA6PuQu3sHsr4C1vyyjUr97l9
|
||||
tvdGlpp20LWtSZQMjHZ3pARYTTsTHTeY3DgQcRcHNicVKx8k3ZepWeeW9vw+pL+V
|
||||
zSt3RsoVrH6gsCSrfr4sS3aqzX9AbjwQvh48CJ3mLQ1m70kHV+xbZIh1+4pB/hyP
|
||||
1wKyUE18ZkOptXvO/TtoHzLQCecpkXtWzmry1Eh2isvXA+NMrAtLibGsyM1mtm7i
|
||||
5ozevzHabvvCDBEe+KgZdONgVhhhvm2eOd+/s4w3rw4ETud4fI/ZAJyWXhiIKFnA
|
||||
VJbElWruSAoVBW7p2bsF5PbmVzvo8vXL+VylxYD+AQKBgQDhLoRKTVhNkn/QjKxq
|
||||
sdOh+QZra0LzjVpAmkQzu7wZMSHEz9qePQciDQQrYKrmRF1vNcIRCVUTqWYheJ/1
|
||||
lKRrCGa0ab6k96zkWMqLHD5u+UeJV7r1dJIx08ME9kNJ+x/XtB8klRIji16NiQUS
|
||||
qc6p8z0M2AnbJzsRfWZRH8FeYwKBgQDHu8dzdtVGI7MtxfPOE/bfajiopDg8BdTC
|
||||
pdug2T8XofRHRq7Q+0vYjTAZFT/slib91Pk6VvvPdo9VBZiL4omv4dAq6mOOdX/c
|
||||
U14mJe1X5GCrr8ExZ8BfNJ3t/6sV1fcxyJwAw7iBguqxA2JqdM/wFk10K8XqvzVn
|
||||
CD6O9yGt2QKBgFX1BMi8N538809vs41S7l9hCQNOQZNo/O+2M5yv6ECRkbtoQKKw
|
||||
1x03bMUGNJaLuELweXE5Z8GGo5bZTe5X3F+DKHlr+DtO1C+ieUaa9HY2MAmMdLCn
|
||||
2/qrREGLo+oEs4YKmuzC/taUp/ZNPKOAMISNdluFyFVg51pozPrgrVbTAoGBAKkE
|
||||
LBl3O67o0t0vH8sJdeVFG8EJhlS0koBMnfgVHqC++dm+5HwPyvTrNQJkyv1HaqNt
|
||||
r6FArkG3ED9gRuBIyT6+lctbIPgSUip9mbQqcBfqOCvQxGksZMur2ODncz09HLtS
|
||||
CUFUXjOqNzOnq4ZuZu/Bz7U4vXiSaXxQq6+LTUKxAoGAFZU/qrI06XxnrE9A1X0W
|
||||
l7DSkpZaDcu11NrZ473yONih/xOZNh4SSBpX8a7F6Pmh9BdtGqphML8NFPvQKcfP
|
||||
b9H2iid2tc292uyrUEb5uTMmv61zoTwtitqLzO0+tS6PT3fXobX+eyeEWKzPBljL
|
||||
HFtxG5CCXpkdnWRmaJnhTzA=
|
||||
-----END PRIVATE KEY-----
|
@ -0,0 +1,7 @@
|
||||
version: '2.2'
|
||||
services:
|
||||
php1:
|
||||
build:
|
||||
context: ./
|
||||
# to keep container running
|
||||
command: sleep infinity
|
@ -0,0 +1,27 @@
|
||||
<?php
|
||||
$host = $argv[1];
|
||||
$db = "system";
|
||||
$user = $argv[3];
|
||||
$pass = $argv[4];
|
||||
$charset = "utf8mb4";
|
||||
$port = $argv[2];
|
||||
|
||||
$dsn = "mysql:host=$host;port=$port;dbname=$db;charset=$charset";
|
||||
$options = [
|
||||
PDO::ATTR_ERRMODE => PDO::ERRMODE_EXCEPTION,
|
||||
PDO::ATTR_DEFAULT_FETCH_MODE => PDO::FETCH_ASSOC,
|
||||
PDO::ATTR_EMULATE_PREPARES => false,
|
||||
PDO::MYSQL_ATTR_DIRECT_QUERY => true,
|
||||
PDO::MYSQL_ATTR_SSL_CERT => "client.crt",
|
||||
PDO::MYSQL_ATTR_SSL_KEY => "client.key",
|
||||
PDO::MYSQL_ATTR_SSL_VERIFY_SERVER_CERT => 0,
|
||||
];
|
||||
$pdo = new PDO($dsn, $user, $pass, $options);
|
||||
|
||||
$stmt = $pdo->query("SELECT name FROM tables WHERE name = 'tables'");
|
||||
|
||||
foreach ($stmt as $row)
|
||||
{
|
||||
echo $row["name"] . "\n";
|
||||
}
|
||||
?>
|
@ -43,6 +43,13 @@ def golang_container():
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_golang1_1')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def php_container():
|
||||
docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'php-mysqlnd', 'docker_compose.yml')
|
||||
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_php1_1')
|
||||
|
||||
|
||||
def test_mysql_client(mysql_client, server_address):
|
||||
# type: (Container, str) -> None
|
||||
code, (stdout, stderr) = mysql_client.exec_run('''
|
||||
@ -138,3 +145,10 @@ def test_golang_client(server_address, golang_container):
|
||||
with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp:
|
||||
reference = fp.read()
|
||||
assert stdout == reference
|
||||
|
||||
|
||||
def test_php_client(server_address, php_container):
|
||||
# type: (str, Container) -> None
|
||||
code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout == 'tables\n'
|
||||
|
@ -0,0 +1,2 @@
|
||||
1
|
||||
1
|
23
dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh
Executable file
23
dbms/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh
Executable file
@ -0,0 +1,23 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CUR_DIR/../shell_config.sh
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test_optimize_exception (date Date) ENGINE=MergeTree() PARTITION BY toYYYYMM(date) ORDER BY date"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test_optimize_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/optimize', 'r1') PARTITION BY toYYYYMM(date) ORDER BY date"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test_optimize_exception VALUES (toDate('2017-09-09')), (toDate('2017-09-10'))"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test_optimize_exception VALUES (toDate('2017-09-09')), (toDate('2017-09-10'))"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test_optimize_exception_replicated VALUES (toDate('2017-09-09')), (toDate('2017-09-10'))"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test_optimize_exception_replicated VALUES (toDate('2017-09-09')), (toDate('2017-09-10'))"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --query="OPTIMIZE TABLE test_optimize_exception PARTITION 201709 FINAL"
|
||||
${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --query="OPTIMIZE TABLE test_optimize_exception_replicated PARTITION 201709 FINAL"
|
||||
|
||||
echo `${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --server_logs_file=/dev/null --query="OPTIMIZE TABLE test_optimize_exception PARTITION 201710" 2>&1` \
|
||||
| grep -c 'Code: 388. DB::Exception: .* DB::Exception: .* Cannot select parts for optimization'
|
||||
echo `${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --server_logs_file=/dev/null --query="OPTIMIZE TABLE test_optimize_exception_replicated PARTITION 201710" 2>&1` \
|
||||
| grep -c 'Code: 388. DB::Exception: .* DB::Exception:.* Cannot select parts for optimization'
|
@ -8,16 +8,16 @@ def gen_queries():
|
||||
columns = tuple('a b c d'.split())
|
||||
order_by_columns = tuple('a b c'.split())
|
||||
partition_by_columns = tuple(' tuple() a'.split())
|
||||
|
||||
|
||||
for partition in partition_by_columns:
|
||||
for key_mask in range(1, 1 << len(order_by_columns)):
|
||||
key = ','.join(order_by_columns[i] for i in range(len(order_by_columns)) if (1 << i) & key_mask != 0)
|
||||
create_query = create_template.format(key, partition)
|
||||
for q in (drop_query, create_query, insert_query):
|
||||
yield q
|
||||
|
||||
|
||||
for column, value in zip(columns, values):
|
||||
yield 'select {} in {} from tab_00386'.format(column, value)
|
||||
yield 'select {} in {} from tab_00386'.format(column, value)
|
||||
yield 'select {} in tuple({}) from tab_00386'.format(column, value)
|
||||
yield 'select {} in (select {} from tab_00386) from tab_00386'.format(column, column)
|
||||
|
||||
@ -26,7 +26,7 @@ def gen_queries():
|
||||
yield 'select ({}, {}) in tuple({}, {}) from tab_00386'.format(columns[i], columns[j], values[i], values[j])
|
||||
yield 'select ({}, {}) in (select {}, {} from tab_00386) from tab_00386'.format(columns[i], columns[j], columns[i], columns[j])
|
||||
yield 'select ({}, {}) in (select ({}, {}) from tab_00386) from tab_00386'.format(columns[i], columns[j], columns[i], columns[j])
|
||||
|
||||
|
||||
yield "select e in (1, 'a') from tab_00386"
|
||||
yield "select f in tuple((1, 'a')) from tab_00386"
|
||||
yield "select f in tuple(tuple((1, 'a'))) from tab_00386"
|
||||
@ -41,7 +41,7 @@ import os
|
||||
|
||||
def main():
|
||||
url = os.environ['CLICKHOUSE_URL_PARAMS']
|
||||
|
||||
|
||||
for q in gen_queries():
|
||||
resp = requests.post(url, data=q)
|
||||
if resp.status_code != 200 or resp.text.strip() not in ('1', ''):
|
||||
|
@ -0,0 +1,24 @@
|
||||
100 10
|
||||
120 8
|
||||
0 0
|
||||
1 1
|
||||
0 0
|
||||
1 1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
2
|
||||
3
|
||||
2
|
||||
4
|
||||
3
|
27
dbms/tests/queries/0_stateless/00969_columns_clause.sql
Normal file
27
dbms/tests/queries/0_stateless/00969_columns_clause.sql
Normal file
@ -0,0 +1,27 @@
|
||||
DROP TABLE IF EXISTS ColumnsClauseTest;
|
||||
|
||||
CREATE TABLE ColumnsClauseTest (product_price Int64, product_weight Int16, amount Int64) Engine=TinyLog;
|
||||
INSERT INTO ColumnsClauseTest VALUES (100, 10, 324), (120, 8, 23);
|
||||
SELECT COLUMNS('product.*') from ColumnsClauseTest ORDER BY product_price;
|
||||
|
||||
DROP TABLE ColumnsClauseTest;
|
||||
|
||||
SELECT number, COLUMNS('') FROM numbers(2);
|
||||
SELECT number, COLUMNS('ber') FROM numbers(2); -- It works for unanchored regular expressions.
|
||||
SELECT number, COLUMNS('x') FROM numbers(2);
|
||||
SELECT COLUMNS('') FROM numbers(2);
|
||||
|
||||
SELECT COLUMNS('x') FROM numbers(10) WHERE number > 5; -- { serverError 51 }
|
||||
|
||||
SELECT * FROM numbers(2) WHERE NOT ignore();
|
||||
SELECT * FROM numbers(2) WHERE NOT ignore(*);
|
||||
SELECT * FROM numbers(2) WHERE NOT ignore(COLUMNS('.+'));
|
||||
SELECT * FROM numbers(2) WHERE NOT ignore(COLUMNS('x'));
|
||||
SELECT COLUMNS('n') + COLUMNS('u') FROM system.numbers LIMIT 2;
|
||||
|
||||
SELECT COLUMNS('n') + COLUMNS('u') FROM (SELECT 1 AS a, 2 AS b); -- { serverError 42 }
|
||||
SELECT COLUMNS('a') + COLUMNS('b') FROM (SELECT 1 AS a, 2 AS b);
|
||||
SELECT COLUMNS('a') + COLUMNS('a') FROM (SELECT 1 AS a, 2 AS b);
|
||||
SELECT COLUMNS('b') + COLUMNS('b') FROM (SELECT 1 AS a, 2 AS b);
|
||||
SELECT COLUMNS('a|b') + COLUMNS('b') FROM (SELECT 1 AS a, 2 AS b); -- { serverError 42 }
|
||||
SELECT plus(COLUMNS('^(a|b)$')) FROM (SELECT 1 AS a, 2 AS b);
|
@ -72,7 +72,7 @@ For a description of request parameters, see [request description](../../query_l
|
||||
It must depends on `Date` or `DateTime` column and has one `Date` or `DateTime` column as a result. Example:
|
||||
`TTL date + INTERVAL 1 DAY`
|
||||
|
||||
For more details, see [TTL for columns and tables](mergetree.md)
|
||||
For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl)
|
||||
|
||||
- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`:
|
||||
- `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. By default, 8192. The list of all available parameters you can see in [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h).
|
||||
|
Loading…
Reference in New Issue
Block a user