mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse into compression_codec_delta
This commit is contained in:
commit
37c78aa3d2
@ -5,6 +5,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/hex.h>
|
||||
#include <sstream>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -20,14 +21,16 @@ extern const int CORRUPTED_DATA;
|
||||
CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs)
|
||||
: codecs(codecs)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
for (size_t idx = 0; idx < codecs.size(); idx++)
|
||||
{
|
||||
if (idx != 0)
|
||||
codec_desc = codec_desc + ',';
|
||||
ss << ',' << ' ';
|
||||
|
||||
const auto codec = codecs[idx];
|
||||
codec_desc = codec_desc + codec->getCodecDesc();
|
||||
ss << codec->getCodecDesc();
|
||||
}
|
||||
codec_desc = ss.str();
|
||||
}
|
||||
|
||||
UInt8 CompressionCodecMultiple::getMethodByte() const
|
||||
|
@ -103,7 +103,7 @@ private:
|
||||
void visit(const ASTIdentifier & identifier, ASTPtr & ast) const
|
||||
{
|
||||
if (ast->children.empty())
|
||||
ast = createDatabaseAndTableNode(database_name, identifier.name);
|
||||
ast = createTableIdentifier(database_name, identifier.name);
|
||||
}
|
||||
|
||||
void visit(ASTSubquery & subquery, ASTPtr &) const
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
* instead of doing a subquery, you just need to read it.
|
||||
*/
|
||||
|
||||
auto database_and_table_name = createDatabaseAndTableNode("", external_table_name);
|
||||
auto database_and_table_name = createTableIdentifier("", external_table_name);
|
||||
|
||||
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -167,8 +168,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
|
||||
std::string table;
|
||||
std::tie(database, table) = getRemoteDatabaseAndTableName(*storage);
|
||||
|
||||
/// TODO: find a way to avoid AST node replacing
|
||||
database_and_table = createDatabaseAndTableNode(database, table);
|
||||
database_and_table = createTableIdentifier(database, table);
|
||||
}
|
||||
else
|
||||
throw Exception("InJoinSubqueriesPreprocessor: unexpected value of 'distributed_product_mode' setting", ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -39,6 +39,19 @@ void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||
writeString(name, ostr);
|
||||
}
|
||||
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name)
|
||||
{
|
||||
if (database_name.empty())
|
||||
return ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
database_and_table->children = {database, table};
|
||||
return database_and_table;
|
||||
}
|
||||
|
||||
bool isIdentifier(const IAST * const ast)
|
||||
{
|
||||
if (ast)
|
||||
@ -67,7 +80,7 @@ bool getIdentifierName(const ASTPtr & ast, String & name)
|
||||
|
||||
std::optional<String> getColumnIdentifierName(const ASTIdentifier & node)
|
||||
{
|
||||
if (!node.special())
|
||||
if (!node.special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
@ -76,14 +89,14 @@ std::optional<String> getColumnIdentifierName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (!id->special())
|
||||
if (!id->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
||||
std::optional<String> getTableIdentifierName(const ASTIdentifier & node)
|
||||
{
|
||||
if (node.special())
|
||||
if (node.special)
|
||||
return node.name;
|
||||
return {};
|
||||
}
|
||||
@ -92,7 +105,7 @@ std::optional<String> getTableIdentifierName(const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
if (auto id = typeid_cast<const ASTIdentifier *>(ast.get()))
|
||||
if (id->special())
|
||||
if (id->special)
|
||||
return id->name;
|
||||
return {};
|
||||
}
|
||||
|
@ -11,18 +11,16 @@ namespace DB
|
||||
/// Identifier (column, table or alias)
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
enum Kind /// TODO This is semantic, not syntax. Remove it.
|
||||
{
|
||||
General,
|
||||
Special, // Database, Table, Format
|
||||
};
|
||||
|
||||
public:
|
||||
/// name. The composite identifier here will have a concatenated name (of the form a.b.c), and individual components will be available inside the children.
|
||||
String name;
|
||||
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = General)
|
||||
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
|
||||
ASTIdentifier(const String & name_)
|
||||
: name(name_)
|
||||
, special(false)
|
||||
{
|
||||
range = StringRange(name.data(), name.data() + name.size());
|
||||
}
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char delim) const override { return "Identifier" + (delim + name); }
|
||||
@ -34,31 +32,35 @@ public:
|
||||
set.insert(name);
|
||||
}
|
||||
|
||||
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
|
||||
{
|
||||
return std::make_shared<ASTIdentifier>(name_, ASTIdentifier::Special);
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
|
||||
private:
|
||||
Kind kind;
|
||||
bool special; /// TODO: it would be ptr to semantic here
|
||||
|
||||
void setSpecial() { kind = Special; }
|
||||
bool special() const { return kind == Special; }
|
||||
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
|
||||
{
|
||||
auto ret = std::make_shared<ASTIdentifier>(name_);
|
||||
ret->special = true;
|
||||
return ret;
|
||||
}
|
||||
|
||||
void setSpecial() { special = true; }
|
||||
|
||||
friend void setIdentifierSpecial(ASTPtr &);
|
||||
friend std::optional<String> getColumnIdentifierName(const ASTIdentifier & node);
|
||||
friend std::optional<String> getColumnIdentifierName(const ASTPtr & ast);
|
||||
friend std::optional<String> getTableIdentifierName(const ASTIdentifier & node);
|
||||
friend std::optional<String> getTableIdentifierName(const ASTPtr & ast);
|
||||
friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
};
|
||||
|
||||
|
||||
/// ASTIdentifier Helpers: hide casts and semantic.
|
||||
|
||||
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
|
||||
|
||||
bool isIdentifier(const IAST * const ast);
|
||||
inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); }
|
||||
|
||||
|
@ -17,19 +17,6 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
ASTPtr createDatabaseAndTableNode(const String & database_name, const String & table_name)
|
||||
{
|
||||
if (database_name.empty())
|
||||
return ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
database_and_table->children = {database, table};
|
||||
return database_and_table;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTSelectQuery::clone() const
|
||||
{
|
||||
@ -338,7 +325,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
|
||||
}
|
||||
|
||||
String table_alias = getTableExpressionAlias(table_expression);
|
||||
table_expression->database_and_table_name = createDatabaseAndTableNode(database_name, table_name);
|
||||
table_expression->database_and_table_name = createTableIdentifier(database_name, table_name);
|
||||
|
||||
if (!table_alias.empty())
|
||||
table_expression->database_and_table_name->setAlias(table_alias);
|
||||
|
@ -52,7 +52,4 @@ protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
|
||||
ASTPtr createDatabaseAndTableNode(const String & database_name, const String & table_name);
|
||||
|
||||
}
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -30,6 +31,7 @@ namespace ErrorCodes
|
||||
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast)
|
||||
{
|
||||
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
|
||||
const CompressionCodecFactory & compression_codec_factory = CompressionCodecFactory::instance();
|
||||
|
||||
if (command_ast->type == ASTAlterCommand::ADD_COLUMN)
|
||||
{
|
||||
@ -49,6 +51,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.default_expression = ast_col_decl.default_expression;
|
||||
}
|
||||
|
||||
if (ast_col_decl.codec)
|
||||
command.codec = compression_codec_factory.get(ast_col_decl.codec);
|
||||
|
||||
if (command_ast->column)
|
||||
command.after_column = *getIdentifierName(command_ast->column);
|
||||
|
||||
@ -86,6 +91,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.default_expression = ast_col_decl.default_expression;
|
||||
}
|
||||
|
||||
if (ast_col_decl.codec)
|
||||
command.codec = compression_codec_factory.get(ast_col_decl.codec);
|
||||
|
||||
if (ast_col_decl.comment)
|
||||
{
|
||||
const auto & ast_comment = typeid_cast<ASTLiteral &>(*ast_col_decl.comment);
|
||||
@ -168,6 +176,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
||||
if (default_expression)
|
||||
columns_description.defaults.emplace(column_name, ColumnDefault{default_kind, default_expression});
|
||||
|
||||
if (codec)
|
||||
columns_description.codecs.emplace(column_name, codec);
|
||||
|
||||
/// Slow, because each time a list is copied
|
||||
columns_description.ordinary = Nested::flatten(columns_description.ordinary);
|
||||
}
|
||||
@ -200,6 +211,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
||||
}
|
||||
else if (type == MODIFY_COLUMN)
|
||||
{
|
||||
if (codec)
|
||||
columns_description.codecs[column_name] = codec;
|
||||
|
||||
if (!is_mutable())
|
||||
{
|
||||
auto & comments = columns_description.comments;
|
||||
|
@ -55,6 +55,9 @@ struct AlterCommand
|
||||
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
|
||||
bool ignore = false;
|
||||
|
||||
/// For ADD and MODIFY
|
||||
CompressionCodecPtr codec;
|
||||
|
||||
AlterCommand() = default;
|
||||
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
|
||||
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,
|
||||
|
@ -41,6 +41,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
{ "is_in_sorting_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sampling_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "compression_codec", std::make_shared<DataTypeString>() },
|
||||
}));
|
||||
}
|
||||
|
||||
@ -86,6 +87,7 @@ protected:
|
||||
NamesAndTypesList columns;
|
||||
ColumnDefaults column_defaults;
|
||||
ColumnComments column_comments;
|
||||
ColumnCodecs column_codecs;
|
||||
Names cols_required_for_partition_key;
|
||||
Names cols_required_for_sorting_key;
|
||||
Names cols_required_for_primary_key;
|
||||
@ -114,6 +116,7 @@ protected:
|
||||
}
|
||||
|
||||
columns = storage->getColumns().getAll();
|
||||
column_codecs = storage->getColumns().codecs;
|
||||
column_defaults = storage->getColumns().defaults;
|
||||
column_comments = storage->getColumns().comments;
|
||||
|
||||
@ -219,6 +222,20 @@ protected:
|
||||
res_columns[res_index++]->insert(find_in_vector(cols_required_for_sampling));
|
||||
}
|
||||
|
||||
{
|
||||
const auto it = column_codecs.find(column.name);
|
||||
if (it == std::end(column_codecs))
|
||||
{
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert("CODEC(" + it->second->getCodecDesc() + ")");
|
||||
}
|
||||
}
|
||||
|
||||
++rows_count;
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,33 @@
|
||||
2018-01-01 1
|
||||
2018-01-01 2
|
||||
CODEC(ZSTD)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
2018-01-01 4 4
|
||||
CODEC(NONE)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
|
||||
2018-01-01 1 default_value
|
||||
2018-01-01 2 default_value
|
||||
2018-01-01 3 3
|
||||
2018-01-01 4 4
|
||||
2018-01-01 5 5
|
||||
2018-01-01 6 6
|
||||
2018-01-01 7 7
|
||||
2018-01-01 8 8
|
||||
CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE)
|
||||
CODEC(NONE, LZ4, LZ4HC, ZSTD)
|
||||
2
|
||||
1
|
@ -0,0 +1,88 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.alter_compression_codec;
|
||||
|
||||
CREATE TABLE test.alter_compression_codec (
|
||||
somedate Date CODEC(LZ4),
|
||||
id UInt64 CODEC(NONE)
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id;
|
||||
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 1);
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 2);
|
||||
SELECT * FROM test.alter_compression_codec ORDER BY id;
|
||||
|
||||
ALTER TABLE test.alter_compression_codec ADD COLUMN alter_column String DEFAULT 'default_value' CODEC(ZSTD);
|
||||
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 3, '3');
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 4, '4');
|
||||
SELECT * FROM test.alter_compression_codec ORDER BY id;
|
||||
|
||||
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column CODEC(NONE);
|
||||
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 5, '5');
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 6, '6');
|
||||
SELECT * FROM test.alter_compression_codec ORDER BY id;
|
||||
|
||||
OPTIMIZE TABLE test.alter_compression_codec FINAL;
|
||||
SELECT * FROM test.alter_compression_codec ORDER BY id;
|
||||
|
||||
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
|
||||
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 7, '7');
|
||||
INSERT INTO test.alter_compression_codec VALUES('2018-01-01', 8, '8');
|
||||
OPTIMIZE TABLE test.alter_compression_codec FINAL;
|
||||
SELECT * FROM test.alter_compression_codec ORDER BY id;
|
||||
|
||||
ALTER TABLE test.alter_compression_codec MODIFY COLUMN alter_column FixedString(100);
|
||||
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec' AND name = 'alter_column';
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS test.alter_compression_codec;
|
||||
|
||||
DROP TABLE IF EXISTS test.alter_bad_codec;
|
||||
|
||||
CREATE TABLE test.alter_bad_codec (
|
||||
somedate Date CODEC(LZ4),
|
||||
id UInt64 CODEC(NONE)
|
||||
) ENGINE = MergeTree() ORDER BY tuple();
|
||||
|
||||
ALTER TABLE test.alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(gbdgkjsdh); -- { serverError 432 }
|
||||
|
||||
ALTER TABLE test.alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 }
|
||||
|
||||
DROP TABLE IF EXISTS test.alter_bad_codec;
|
||||
|
||||
DROP TABLE IF EXISTS test.large_alter_table;
|
||||
DROP TABLE IF EXISTS test.store_of_hash;
|
||||
|
||||
CREATE TABLE test.large_alter_table (
|
||||
somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)),
|
||||
id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC),
|
||||
data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4)
|
||||
) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2;
|
||||
|
||||
INSERT INTO test.large_alter_table SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000;
|
||||
|
||||
CREATE TABLE test.store_of_hash (hash UInt64) ENGINE = Memory();
|
||||
|
||||
INSERT INTO test.store_of_hash SELECT sum(cityHash64(*)) FROM test.large_alter_table;
|
||||
|
||||
ALTER TABLE test.large_alter_table MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD);
|
||||
|
||||
OPTIMIZE TABLE test.large_alter_table;
|
||||
|
||||
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'large_alter_table' AND name = 'data';
|
||||
|
||||
DETACH TABLE test.large_alter_table;
|
||||
ATTACH TABLE test.large_alter_table;
|
||||
|
||||
INSERT INTO test.store_of_hash SELECT sum(cityHash64(*)) FROM test.large_alter_table;
|
||||
|
||||
SELECT COUNT(hash) FROM test.store_of_hash;
|
||||
SELECT COUNT(DISTINCT hash) FROM test.store_of_hash;
|
||||
|
||||
DROP TABLE IF EXISTS test.large_alter_table;
|
||||
DROP TABLE IF EXISTS test.store_of_hash;
|
@ -26,52 +26,61 @@ SQL-style and C-style comments are supported.
|
||||
SQL-style comments: from `--` to the end of the line. The space after `--` can be omitted.
|
||||
Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either.
|
||||
|
||||
## Keywords
|
||||
## Keywords {#syntax-keywords}
|
||||
|
||||
Keywords (such as `SELECT`) are not case-sensitive. Everything else (column names, functions, and so on), in contrast to standard SQL, is case-sensitive. Keywords are not reserved (they are just parsed as keywords in the corresponding context).
|
||||
Keywords (such as `SELECT`) are not case-sensitive. Everything else (column names, functions, and so on), in contrast to standard SQL, is case-sensitive.
|
||||
|
||||
## Identifiers
|
||||
Keywords are not reserved (they are just parsed as keywords in the corresponding context). If you use [identifiers](#syntax-identifiers) the same as the keywords, enclose them into quotes. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`.
|
||||
|
||||
Identifiers (column names, functions, and data types) can be quoted or non-quoted.
|
||||
Non-quoted identifiers start with a Latin letter or underscore, and continue with a Latin letter, underscore, or number. In other words, they must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$`. Examples: `x, _1, X_y__Z123_.`
|
||||
## Identifiers {#syntax-identifiers}
|
||||
|
||||
Quoted identifiers are placed in reversed quotation marks `` `id` `` (the same as in MySQL), and can indicate any set of bytes (non-empty). In addition, symbols (for example, the reverse quotation mark) inside this type of identifier can be backslash-escaped. Escaping rules are the same as for string literals (see below).
|
||||
We recommend using identifiers that do not need to be quoted.
|
||||
Identifiers are:
|
||||
|
||||
- Cluster, database, table, partition and column names;
|
||||
- Functions;
|
||||
- Data types;
|
||||
- [Expression aliases](#syntax-expression_aliases).
|
||||
|
||||
Identifiers can be quoted or non-quoted. It is recommended to use non-quoted identifiers.
|
||||
|
||||
Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x, _1, X_y__Z123_.`
|
||||
|
||||
If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``.
|
||||
|
||||
## Literals
|
||||
|
||||
There are numeric literals, string literals, and compound literals.
|
||||
There are: numeric, string, compound and `NULL` literals.
|
||||
|
||||
### Numeric Literals
|
||||
|
||||
### Numeric
|
||||
|
||||
A numeric literal tries to be parsed:
|
||||
|
||||
- First as a 64-bit signed number, using the 'strtoull' function.
|
||||
- If unsuccessful, as a 64-bit unsigned number, using the 'strtoll' function.
|
||||
- If unsuccessful, as a floating-point number using the 'strtod' function.
|
||||
- First as a 64-bit signed number, using the [strtoull](https://en.cppreference.com/w/cpp/string/byte/strtoul) function.
|
||||
- If unsuccessful, as a 64-bit unsigned number, using the [strtoll](https://en.cppreference.com/w/cpp/string/byte/strtol) function.
|
||||
- If unsuccessful, as a floating-point number using the [strtod](https://en.cppreference.com/w/cpp/string/byte/strtof) function.
|
||||
- Otherwise, an error is returned.
|
||||
|
||||
The corresponding value will have the smallest type that the value fits in.
|
||||
For example, 1 is parsed as UInt8, but 256 is parsed as UInt16. For more information, see "Data types".
|
||||
For example, 1 is parsed as `UInt8`, but 256 is parsed as `UInt16`. For more information, see [Data types](../data_types/index.md).
|
||||
|
||||
Examples: `1`, `18446744073709551615`, `0xDEADBEEF`, `01`, `0.1`, `1e100`, `-1e-100`, `inf`, `nan`.
|
||||
|
||||
|
||||
### String Literals
|
||||
### String
|
||||
|
||||
Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where "c" is any character, are converted to "c". This means that you can use the sequences `\'`and`\\`. The value will have the String type.
|
||||
Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where `c` is any character, are converted to `c`. This means that you can use the sequences `\'`and`\\`. The value will have the String type.
|
||||
|
||||
The minimum set of characters that you need to escape in string literals: `'` and `\`.
|
||||
The minimum set of characters that you need to escape in string literals: `'` and `\`. Single quote can be escaped with the single quote, literals `'It\'s'` and `'It''s'` are equal.
|
||||
|
||||
### Compound Literals
|
||||
### Compound
|
||||
|
||||
Constructions are supported for arrays: `[1, 2, 3]` and tuples: `(1, 'Hello, world!', 2)`..
|
||||
Actually, these are not literals, but expressions with the array creation operator and the tuple creation operator, respectively.
|
||||
For more information, see the section "Operators2".
|
||||
An array must consist of at least one item, and a tuple must have at least two items.
|
||||
Tuples have a special purpose for use in the IN clause of a SELECT query. Tuples can be obtained as the result of a query, but they can't be saved to a database (with the exception of Memory-type tables).
|
||||
Tuples have a special purpose for use in the `IN` clause of a `SELECT` query. Tuples can be obtained as the result of a query, but they can't be saved to a database (with the exception of [Memory](../operations/table_engines/memory.md) tables).
|
||||
|
||||
### NULL Literal {#null-literal}
|
||||
### NULL {#null-literal}
|
||||
|
||||
Indicates that the value is missing.
|
||||
|
||||
@ -92,13 +101,13 @@ There are regular and aggregate functions (see the section "Aggregate functions"
|
||||
|
||||
Operators are converted to their corresponding functions during query parsing, taking their priority and associativity into account.
|
||||
For example, the expression `1 + 2 * 3 + 4` is transformed to `plus(plus(1, multiply(2, 3)), 4)`.
|
||||
For more information, see the section "Operators" below.
|
||||
|
||||
|
||||
## Data Types and Database Table Engines
|
||||
|
||||
Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an arguments list in brackets. For more information, see the sections "Data types," "Table engines," and "CREATE".
|
||||
|
||||
## Expression Aliases
|
||||
## Expression Aliases {#syntax-expression_aliases}
|
||||
|
||||
Alias is a user defined name for an expression in a query.
|
||||
|
||||
@ -106,27 +115,27 @@ Alias is a user defined name for an expression in a query.
|
||||
expr AS alias
|
||||
```
|
||||
|
||||
- `AS` — keyword for defining aliases. You can define alias for a table name or a column name in SELECT clause skipping `AS` keyword.
|
||||
- `AS` — Keyword for defining aliases. You can define alias for a table name or a column name in the `SELECT` clause skipping `AS` keyword.
|
||||
|
||||
For example, `SELECT b.column_name from t b`.
|
||||
For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`.
|
||||
|
||||
In the [CAST function](functions/type_conversion_functions.md), the `AS` keyword has another meaning. See the description of the function.
|
||||
|
||||
- `expr` — any expression supported by ClickHouse.
|
||||
- `expr` — Any expression supported by ClickHouse.
|
||||
|
||||
For example `SELECT column_name * 2 AS double FROM some_table`.
|
||||
|
||||
- `alias` — [string literal](#string-literals). If an alias contains spaces, enclose it in double quotes or backticks.
|
||||
- `alias` — Name for `expr`. Aliases should comply with the [identifiers](#syntax-identifiers) syntax.
|
||||
|
||||
For example, `SELECT "table t".col_name FROM t AS "table t"`.
|
||||
For example, `SELECT "table t".column_name FROM table_name AS "table t"`.
|
||||
|
||||
### Peculiarities of Use
|
||||
|
||||
Aliases are global for a query or subquery and you can define alias in any part of a query for any expression. For example, `SELECT (1 AS n) + 2, n`.
|
||||
|
||||
Aliases are not visible in between subqueries. For example, while executing the query `SELECT (SELECT sum(b.a) + num FROM b) - a.a AS num FROM a` ClickHouse generates exception `Unknown identifier: num`.
|
||||
Aliases are not visible in subqueries. For example, while executing the query `SELECT (SELECT sum(b.a) + num FROM b) - a.a AS num FROM a` ClickHouse generates the exception `Unknown identifier: num`.
|
||||
|
||||
If alias is defined for result columns in SELECT clause in a subquery, these columns are visible in outer query. For example, `SELECT n + m FROM (SELECT 1 AS n, 2 AS m)`.
|
||||
If an alias is defined for result columns in `SELECT` clause in a subquery, these columns are visible in outer query. For example, `SELECT n + m FROM (SELECT 1 AS n, 2 AS m)`.
|
||||
|
||||
Be careful with aliases the same as column or table names. Let's consider the following example:
|
||||
|
||||
|
@ -245,7 +245,7 @@ TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster]
|
||||
|
||||
Удаляет все данные из таблицы. Если условие `IF EXISTS` не указано, запрос вернет ошибку, если таблицы не существует.
|
||||
|
||||
Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../operations/table_engines/view.md), [File](../operations/table_engines/file.md), [URL](../operations/table_engines/url.md) and [Null](../operations/table_engines/null.md).
|
||||
Запрос `TRUNCATE` не поддерживается для следующих движков: [View](../operations/table_engines/view.md), [File](../operations/table_engines/file.md), [URL](../operations/table_engines/url.md) и [Null](../operations/table_engines/null.md).
|
||||
|
||||
## USE
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user