Merge branch 'master' of github.com:ClickHouse/ClickHouse

This commit is contained in:
Sergei Shtykov 2019-12-06 14:56:06 +03:00
commit 26e8ea28e7
9 changed files with 117 additions and 15 deletions

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 6216cc01a107ce149863411ca29013a224f80343 Subproject commit 2b273bfe9db89429b2040c024484dee0197e48c7

View File

@ -472,6 +472,7 @@ namespace ErrorCodes
extern const int ACCESS_ENTITY_STORAGE_READONLY = 495; extern const int ACCESS_ENTITY_STORAGE_READONLY = 495;
extern const int QUOTA_REQUIRES_CLIENT_KEY = 496; extern const int QUOTA_REQUIRES_CLIENT_KEY = 496;
extern const int NOT_ENOUGH_PRIVILEGES = 497; extern const int NOT_ENOUGH_PRIVILEGES = 497;
extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED = 498;
extern const int KEEPER_EXCEPTION = 999; extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000; extern const int POCO_EXCEPTION = 1000;

View File

@ -288,7 +288,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable
public: public:
/// Called to load or reload an object. /// Called to load or reload an object.
using CreateObjectFunction = std::function<LoadablePtr( using CreateObjectFunction = std::function<LoadablePtr(
const String & /* name */, const ObjectConfig & /* config */, bool config_changed, const LoadablePtr & /* previous_version */)>; const String & /* name */, const ObjectConfig & /* config */, const LoadablePtr & /* previous_version */)>;
LoadingDispatcher( LoadingDispatcher(
const CreateObjectFunction & create_object_function_, const CreateObjectFunction & create_object_function_,
@ -791,14 +791,13 @@ private:
std::pair<LoadablePtr, std::exception_ptr> loadOneObject( std::pair<LoadablePtr, std::exception_ptr> loadOneObject(
const String & name, const String & name,
const ObjectConfig & config, const ObjectConfig & config,
bool config_changed,
LoadablePtr previous_version) LoadablePtr previous_version)
{ {
LoadablePtr new_object; LoadablePtr new_object;
std::exception_ptr new_exception; std::exception_ptr new_exception;
try try
{ {
new_object = create_object(name, config, config_changed, previous_version); new_object = create_object(name, config, previous_version);
} }
catch (...) catch (...)
{ {
@ -918,7 +917,8 @@ private:
/// Use `create_function` to perform the actual loading. /// Use `create_function` to perform the actual loading.
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time /// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
/// and require access to other objects. /// and require access to other objects.
auto [new_object, new_exception] = loadOneObject(name, info->object_config, info->config_changed, info->object); bool need_complete_loading = !info->object || info->config_changed || info->forced_to_reload;
auto [new_object, new_exception] = loadOneObject(name, info->object_config, need_complete_loading ? nullptr : info->object);
if (!new_object && !new_exception) if (!new_object && !new_exception)
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR); throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
@ -1075,7 +1075,7 @@ private:
ExternalLoader::ExternalLoader(const String & type_name_, Logger * log) ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
: config_files_reader(std::make_unique<LoadablesConfigReader>(type_name_, log)) : config_files_reader(std::make_unique<LoadablesConfigReader>(type_name_, log))
, loading_dispatcher(std::make_unique<LoadingDispatcher>( , loading_dispatcher(std::make_unique<LoadingDispatcher>(
std::bind(&ExternalLoader::createObject, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3, std::placeholders::_4), std::bind(&ExternalLoader::createObject, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3),
type_name_, type_name_,
log)) log))
, periodic_updater(std::make_unique<PeriodicUpdater>(*config_files_reader, *loading_dispatcher)) , periodic_updater(std::make_unique<PeriodicUpdater>(*config_files_reader, *loading_dispatcher))
@ -1225,9 +1225,9 @@ void ExternalLoader::addObjectAndLoad(
ExternalLoader::LoadablePtr ExternalLoader::createObject( ExternalLoader::LoadablePtr ExternalLoader::createObject(
const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const
{ {
if (previous_version && !config_changed) if (previous_version)
return previous_version->clone(); return previous_version->clone();
return create(name, *config.config, config.key_in_config); return create(name, *config.config, config.key_in_config);

View File

@ -175,7 +175,7 @@ protected:
private: private:
struct ObjectConfig; struct ObjectConfig;
LoadablePtr createObject(const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const; LoadablePtr createObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const;
class LoadablesConfigReader; class LoadablesConfigReader;
std::unique_ptr<LoadablesConfigReader> config_files_reader; std::unique_ptr<LoadablesConfigReader> config_files_reader;

View File

@ -18,6 +18,7 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR; extern const int SYNTAX_ERROR;
extern const int TOP_AND_LIMIT_TOGETHER; extern const int TOP_AND_LIMIT_TOGETHER;
extern const int WITH_TIES_WITHOUT_ORDER_BY; extern const int WITH_TIES_WITHOUT_ORDER_BY;
extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED;
} }
@ -67,6 +68,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ASTPtr limit_by_expression_list; ASTPtr limit_by_expression_list;
ASTPtr limit_offset; ASTPtr limit_offset;
ASTPtr limit_length; ASTPtr limit_length;
ASTPtr top_length;
ASTPtr settings; ASTPtr settings;
/// WITH expr list /// WITH expr list
@ -92,14 +94,14 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (open_bracket.ignore(pos, expected)) if (open_bracket.ignore(pos, expected))
{ {
if (!num.parse(pos, limit_length, expected)) if (!num.parse(pos, top_length, expected))
return false; return false;
if (!close_bracket.ignore(pos, expected)) if (!close_bracket.ignore(pos, expected))
return false; return false;
} }
else else
{ {
if (!num.parse(pos, limit_length, expected)) if (!num.parse(pos, top_length, expected))
return false; return false;
} }
@ -186,12 +188,12 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
} }
/// This is needed for TOP expression, because it can also use WITH TIES.
bool limit_with_ties_occured = false;
/// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list | LIMIT offset, length BY expr-list /// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list | LIMIT offset, length BY expr-list
if (s_limit.ignore(pos, expected)) if (s_limit.ignore(pos, expected))
{ {
if (limit_length)
throw Exception("Can not use TOP and LIMIT together", ErrorCodes::TOP_AND_LIMIT_TOGETHER);
ParserToken s_comma(TokenType::Comma); ParserToken s_comma(TokenType::Comma);
if (!exp_elem.parse(pos, limit_length, expected)) if (!exp_elem.parse(pos, limit_length, expected))
@ -204,18 +206,30 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
if (s_with_ties.ignore(pos, expected)) if (s_with_ties.ignore(pos, expected))
{
limit_with_ties_occured = true;
select_query->limit_with_ties = true; select_query->limit_with_ties = true;
} }
}
else if (s_offset.ignore(pos, expected)) else if (s_offset.ignore(pos, expected))
{ {
if (!exp_elem.parse(pos, limit_offset, expected)) if (!exp_elem.parse(pos, limit_offset, expected))
return false; return false;
} }
else if (s_with_ties.ignore(pos, expected)) else if (s_with_ties.ignore(pos, expected))
{
limit_with_ties_occured = true;
select_query->limit_with_ties = true; select_query->limit_with_ties = true;
}
if (s_by.ignore(pos, expected)) if (s_by.ignore(pos, expected))
{ {
/// WITH TIES was used alongside LIMIT BY
/// But there are other kind of queries like LIMIT n BY smth LIMIT m WITH TIES which are allowed.
/// So we have to ignore WITH TIES exactly in LIMIT BY state.
if (limit_with_ties_occured)
throw Exception("Can not use WITH TIES alongside LIMIT BY", ErrorCodes::LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED);
limit_by_length = limit_length; limit_by_length = limit_length;
limit_by_offset = limit_offset; limit_by_offset = limit_offset;
limit_length = nullptr; limit_length = nullptr;
@ -224,8 +238,15 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!exp_list.parse(pos, limit_by_expression_list, expected)) if (!exp_list.parse(pos, limit_by_expression_list, expected))
return false; return false;
} }
if (top_length && limit_length)
throw Exception("Can not use TOP and LIMIT together", ErrorCodes::TOP_AND_LIMIT_TOGETHER);
} }
/// Because TOP n in totally equals LIMIT n
if (top_length)
limit_length = top_length;
/// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES] /// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES]
if (s_limit.ignore(pos, expected)) if (s_limit.ignore(pos, expected))
{ {

View File

@ -0,0 +1,14 @@
OK
OK
upyachka a
test b
foo c
bar d
hello x
world x
upyachka a
test b
foo c
bar d
hello x
world x

View File

@ -0,0 +1,14 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g'`
$CLICKHOUSE_CLIENT --query="SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10) ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table ORDER BY a LIMIT 3 WITH TIES BY a" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="SELECT * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 WITH TIES BY Phrase LIMIT 5;" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="SELECT * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 BY Phrase LIMIT 5 WITH TIES"
$CLICKHOUSE_CLIENT --query="SELECT TOP 5 WITH TIES * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 BY Phrase"

View File

@ -0,0 +1,6 @@
12 -> 102
13 -> 103
14 -> -1
12(r) -> 102
13(r) -> 103
14(r) -> 104

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
set -e -o pipefail
# Run the client.
$CLICKHOUSE_CLIENT --multiquery <<'EOF'
DROP DATABASE IF EXISTS dictdb;
CREATE DATABASE dictdb Engine = Ordinary;
CREATE TABLE dictdb.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO dictdb.table VALUES (12, 102, now());
CREATE DICTIONARY dictdb.dict
(
x Int64 DEFAULT -1,
y Int64 DEFAULT -1,
insert_time DateTime
)
PRIMARY KEY x
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table' DB 'dictdb' UPDATE_FIELD 'insert_time'))
LAYOUT(FLAT())
LIFETIME(1);
EOF
$CLICKHOUSE_CLIENT --query "SELECT '12 -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(12))"
$CLICKHOUSE_CLIENT --query "INSERT INTO dictdb.table VALUES (13, 103, now())"
$CLICKHOUSE_CLIENT --query "INSERT INTO dictdb.table VALUES (14, 104, now() - INTERVAL 1 DAY)"
while [ $($CLICKHOUSE_CLIENT --query "SELECT dictGetInt64('dictdb.dict', 'y', toUInt64(13))") = -1 ]
do
sleep 0.5
done
$CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(13))"
$CLICKHOUSE_CLIENT --query "SELECT '14 -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(14))"
$CLICKHOUSE_CLIENT --query "SYSTEM RELOAD DICTIONARY 'dictdb.dict'"
$CLICKHOUSE_CLIENT --query "SELECT '12(r) -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(12))"
$CLICKHOUSE_CLIENT --query "SELECT '13(r) -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(13))"
$CLICKHOUSE_CLIENT --query "SELECT '14(r) -> ', dictGetInt64('dictdb.dict', 'y', toUInt64(14))"
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb"