Merge pull request #2483 from yandex/cast-fix-bad-code

Fixed bad code in implementation of CAST expression.
This commit is contained in:
alexey-milovidov 2018-06-08 00:25:30 +03:00 committed by GitHub
commit 7bae3506c4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 179 additions and 267 deletions

View File

@ -47,7 +47,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToFixedString>();
factory.registerFunction<FunctionToUnixTimestamp>();
factory.registerFunction<FunctionBuilderCast>();
factory.registerFunction<FunctionBuilderCast>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionToUInt8OrZero>();
factory.registerFunction<FunctionToUInt16OrZero>();

View File

@ -65,27 +65,6 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
bool written = false;
if (arguments && !parameters)
{
if (0 == strcmp(name.data(), "CAST"))
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << name;
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
arguments->children.front()->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS "
<< (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_function : "")
<< typeid_cast<const ASTLiteral &>(*arguments->children.back()).value.safeGet<String>()
<< (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << ')'
<< (settings.hilite ? hilite_none : "");
written = true;
}
if (arguments->children.size() == 1)
{
const char * operators[] =

View File

@ -22,6 +22,8 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/queryToString.h>
namespace DB
{
@ -280,107 +282,45 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
const auto begin = pos;
/// Either CAST(expr AS type) or CAST(expr, 'type')
/// The latter will be parsed normally as a function later.
ParserIdentifier id_parser;
ASTPtr expr_node;
ASTPtr type_node;
ASTPtr identifier;
if (!id_parser.parse(pos, identifier, expected))
return false;
const auto & id = typeid_cast<const ASTIdentifier &>(*identifier).name;
/// TODO This is ridiculous. Please get rid of this.
if (id.length() != strlen(name) || 0 != strcasecmp(id.c_str(), name))
if (ParserKeyword("CAST").ignore(pos, expected)
&& ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)
&& ParserExpression().parse(pos, expr_node, expected)
&& ParserKeyword("AS").ignore(pos, expected)
&& ParserIdentifierWithOptionalParameters().parse(pos, type_node, expected)
&& ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
{
/// Parse as a CASE expression.
pos = begin;
return ParserCase{}.parse(pos, node, expected);
/// Convert to canonical representation in functional form: CAST(expr, 'type')
auto type_literal = std::make_shared<ASTLiteral>(queryToString(type_node));
auto expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children.push_back(expr_node);
expr_list_args->children.push_back(std::move(type_literal));
auto func_node = std::make_shared<ASTFunction>();
func_node->name = "CAST";
func_node->arguments = std::move(expr_list_args);
func_node->children.push_back(func_node->arguments);
node = std::move(func_node);
return true;
}
/// Parse as CAST(expression AS type)
ParserExpressionInCastExpression expression_and_type(false);
ASTPtr expr_list_args;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
const auto contents_begin = pos;
ASTPtr first_argument;
if (!expression_and_type.parse(pos, first_argument, expected))
return false;
/// check for subsequent comma ","
if (pos->type != TokenType::Comma)
{
/// CAST(expression AS type)
const auto type = first_argument->tryGetAlias();
if (type.empty())
{
/// there is only one argument and it has no alias
expected.add(pos, "type identifier");
return false;
}
expr_list_args = std::make_shared<ASTExpressionList>();
first_argument->setAlias({});
expr_list_args->children.push_back(first_argument);
expr_list_args->children.emplace_back(std::make_shared<ASTLiteral>(type));
}
else
{
pos = contents_begin;
/// CAST(expression, 'type')
/// Reparse argument list from scratch
ParserExpressionWithOptionalAlias expression{false};
if (!expression.parse(pos, first_argument, expected))
return false;
if (pos->type != TokenType::Comma)
return false;
++pos;
ParserStringLiteral p_type;
ASTPtr type_as_literal;
if (!p_type.parse(pos, type_as_literal, expected))
{
expected.add(pos, "string literal depicting type");
return false;
}
expr_list_args = std::make_shared<ASTExpressionList>();
expr_list_args->children.push_back(first_argument);
expr_list_args->children.push_back(type_as_literal);
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
const auto function_node = std::make_shared<ASTFunction>();
ASTPtr node_holder{function_node};
function_node->name = name;
function_node->arguments = expr_list_args;
function_node->children.push_back(function_node->arguments);
node = node_holder;
return true;
return false;
}
bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
ParserIdentifier id_parser;
ASTPtr identifier;
if (!id_parser.parse(pos, identifier, expected))
if (!ParserKeyword("EXTRACT").ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
@ -636,7 +576,7 @@ bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
const char * ParserAliasBase::restricted_keywords[] =
const char * ParserAlias::restricted_keywords[] =
{
"FROM",
"FINAL",
@ -667,8 +607,7 @@ const char * ParserAliasBase::restricted_keywords[] =
nullptr
};
template <typename ParserIdentifier>
bool ParserAliasImpl<ParserIdentifier>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_as("AS");
ParserIdentifier id_p;
@ -697,9 +636,6 @@ bool ParserAliasImpl<ParserIdentifier>::parseImpl(Pos & pos, ASTPtr & node, Expe
return true;
}
template class ParserAliasImpl<ParserIdentifier>;
template class ParserAliasImpl<ParserTypeInCastExpression>;
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
{
@ -735,53 +671,22 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserParenthesisExpression paren_p;
ParserSubquery subquery_p;
ParserArray array_p;
ParserArrayOfLiterals array_lite_p;
ParserLiteral lit_p;
ParserExtractExpression extract_p;
ParserCastExpression cast_p;
ParserCompoundIdentifier id_p;
ParserAsterisk asterisk_p;
ParserQualifiedAsterisk qualified_asterisk_p;
if (subquery_p.parse(pos, node, expected))
return true;
if (paren_p.parse(pos, node, expected))
return true;
if (array_lite_p.parse(pos, node, expected))
return true;
if (array_p.parse(pos, node, expected))
return true;
if (lit_p.parse(pos, node, expected))
return true;
if (extract_p.parse(pos, node, expected))
return true;
if (cast_p.parse(pos, node, expected))
return true;
if (qualified_asterisk_p.parse(pos, node, expected))
return true;
if (asterisk_p.parse(pos, node, expected))
return true;
if (id_p.parse(pos, node, expected))
return true;
return false;
return ParserSubquery().parse(pos, node, expected)
|| ParserParenthesisExpression().parse(pos, node, expected)
|| ParserArrayOfLiterals().parse(pos, node, expected)
|| ParserArray().parse(pos, node, expected)
|| ParserLiteral().parse(pos, node, expected)
|| ParserExtractExpression().parse(pos, node, expected)
|| ParserCastExpression().parse(pos, node, expected)
|| ParserCase().parse(pos, node, expected)
|| ParserFunction().parse(pos, node, expected)
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|| ParserAsterisk().parse(pos, node, expected)
|| ParserCompoundIdentifier().parse(pos, node, expected);
}
template <typename ParserAlias>
bool ParserWithOptionalAliasImpl<ParserAlias>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!elem_parser->parse(pos, node, expected))
return false;
@ -831,9 +736,6 @@ bool ParserWithOptionalAliasImpl<ParserAlias>::parseImpl(Pos & pos, ASTPtr & nod
return true;
}
template class ParserWithOptionalAliasImpl<ParserAlias>;
template class ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{

View File

@ -91,20 +91,15 @@ protected:
class ParserCastExpression : public IParserBase
{
/// this name is used for identifying CAST expression among other function calls
static constexpr auto name = "CAST";
protected:
const char * getName() const override { return name; }
const char * getName() const override { return "CAST expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserExtractExpression : public IParserBase
{
static constexpr auto name = "EXTRACT";
protected:
const char * getName() const override { return name; }
const char * getName() const override { return "EXTRACT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
@ -174,18 +169,14 @@ protected:
/** The alias is the identifier before which `AS` comes. For example: AS x_yz123.
*/
struct ParserAliasBase
{
static const char * restricted_keywords[];
};
template <typename ParserIdentifier>
class ParserAliasImpl : public IParserBase, ParserAliasBase
class ParserAlias : public IParserBase
{
public:
ParserAliasImpl(bool allow_alias_without_as_keyword_)
ParserAlias(bool allow_alias_without_as_keyword_)
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {}
protected:
private:
static const char * restricted_keywords[];
bool allow_alias_without_as_keyword;
const char * getName() const { return "alias"; }
@ -193,15 +184,6 @@ protected:
};
class ParserTypeInCastExpression;
extern template class ParserAliasImpl<ParserIdentifier>;
extern template class ParserAliasImpl<ParserTypeInCastExpression>;
using ParserAlias = ParserAliasImpl<ParserIdentifier>;
using ParserCastExpressionAlias = ParserAliasImpl<ParserTypeInCastExpression>;
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
*/
class ParserExpressionElement : public IParserBase
@ -214,11 +196,10 @@ protected:
/** An expression element, possibly with an alias, if appropriate.
*/
template <typename ParserAlias>
class ParserWithOptionalAliasImpl : public IParserBase
class ParserWithOptionalAlias : public IParserBase
{
public:
ParserWithOptionalAliasImpl(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false)
ParserWithOptionalAlias(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false)
: elem_parser(std::move(elem_parser_)), allow_alias_without_as_keyword(allow_alias_without_as_keyword_),
prefer_alias_to_column_name(prefer_alias_to_column_name_) {}
protected:
@ -230,12 +211,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
extern template class ParserWithOptionalAliasImpl<ParserAlias>;
extern template class ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
using ParserWithOptionalAlias = ParserWithOptionalAliasImpl<ParserAlias>;
using ParserCastExpressionWithOptionalAlias = ParserWithOptionalAliasImpl<ParserCastExpressionAlias>;
/** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
* and optionally, NULLS LAST|FIRST

View File

@ -535,13 +535,6 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_
}
ParserExpressionInCastExpression::ParserExpressionInCastExpression(bool allow_alias_without_as_keyword)
: impl(std::make_unique<ParserCastExpressionWithOptionalAlias>(std::make_unique<ParserExpression>(),
allow_alias_without_as_keyword, false))
{
}
bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(

View File

@ -335,22 +335,6 @@ protected:
};
class ParserExpressionInCastExpression : public IParserBase
{
public:
ParserExpressionInCastExpression(bool allow_alias_without_as_keyword);
protected:
ParserPtr impl;
const char * getName() const { return "expression in CAST expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return impl->parse(pos, node, expected);
}
};
/** A comma-separated list of expressions, probably empty. */
class ParserExpressionList : public IParserBase
{

View File

@ -17,14 +17,11 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_end{ "END"};
ParserExpressionWithOptionalAlias p_expr{false};
if (!s_case.parse(pos, node, expected))
{
/// Parse as a simple ASTFunction.
return ParserFunction{}.parse(pos, node, expected);
}
if (!s_case.ignore(pos, expected))
return false;
auto old_pos = pos;
bool has_case_expr = !s_when.parse(pos, node, expected);
bool has_case_expr = !s_when.ignore(pos, expected);
pos = old_pos;
ASTs args;
@ -32,7 +29,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto parse_branches = [&]()
{
bool has_branch = false;
while (s_when.parse(pos, node, expected))
while (s_when.ignore(pos, expected))
{
has_branch = true;
@ -41,7 +38,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
args.push_back(expr_when);
if (!s_then.parse(pos, node, expected))
if (!s_then.ignore(pos, expected))
return false;
ASTPtr expr_then;
@ -53,7 +50,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!has_branch)
return false;
if (!s_else.parse(pos, node, expected))
if (!s_else.ignore(pos, expected))
return false;
ASTPtr expr_else;
@ -61,7 +58,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
args.push_back(expr_else);
if (!s_end.parse(pos, node, expected))
if (!s_end.ignore(pos, expected))
return false;
return true;

View File

@ -78,18 +78,6 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
return false;
}
bool ParserTypeInCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (ParserIdentifierWithOptionalParameters().parse(pos, node, expected))
{
const auto & id_with_params = typeid_cast<const ASTFunction &>(*node);
node = std::make_shared<ASTIdentifier>(String{ id_with_params.range.first, id_with_params.range.second });
return true;
}
return false;
}
bool ParserNameTypePairList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserNameTypePair>(), std::make_unique<ParserToken>(TokenType::Comma), false)

View File

@ -48,13 +48,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
class ParserTypeInCastExpression : public IParserBase
{
protected:
const char * getName() const { return "type in cast expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
template <typename NameParser>
class IParserNameTypePair : public IParserBase

View File

@ -67,9 +67,9 @@ String ColumnsDescription::toString() const
{
WriteBufferFromOwnString buf;
writeString("columns format version: 1\n", buf);
writeCString("columns format version: 1\n", buf);
writeText(ordinary.size() + materialized.size() + aliases.size(), buf);
writeString(" columns:\n", buf);
writeCString(" columns:\n", buf);
const auto write_columns = [this, &buf] (const NamesAndTypesList & columns)
{
@ -79,7 +79,7 @@ String ColumnsDescription::toString() const
writeBackQuotedString(column.name, buf);
writeChar(' ', buf);
writeString(column.type->getName(), buf);
writeText(column.type->getName(), buf);
if (it == std::end(defaults))
{
writeChar('\n', buf);
@ -88,9 +88,9 @@ String ColumnsDescription::toString() const
else
writeChar('\t', buf);
writeString(DB::toString(it->second.kind), buf);
writeText(DB::toString(it->second.kind), buf);
writeChar('\t', buf);
writeString(queryToString(it->second.expression), buf);
writeText(queryToString(it->second.expression), buf);
writeChar('\n', buf);
}
};
@ -123,7 +123,7 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
assertChar(' ', buf);
String type_name;
readString(type_name, buf);
readText(type_name, buf);
auto type = data_type_factory.get(type_name);
if (*buf.position() == '\n')
{
@ -135,7 +135,7 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
assertChar('\t', buf);
String default_kind_str;
readString(default_kind_str, buf);
readText(default_kind_str, buf);
const auto default_kind = columnDefaultKindFromString(default_kind_str);
assertChar('\t', buf);

View File

@ -32,7 +32,7 @@ key UInt64 MATERIALIZED 0 * rand()
hello clickhouse 16
some string 11
payload String
payload_length UInt16 DEFAULT CAST(length(payload) AS UInt16)
payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\')
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload String

View File

@ -1,24 +1,24 @@
d Date DEFAULT CAST(\'2015-12-29\' AS Date)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus *
d Date DEFAULT CAST(\'2015-12-29\' AS Date)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus *
2015-12-29 0 ! plus b
d Date DEFAULT CAST(\'2015-12-29\' AS Date)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum16(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus no letter
2015-12-29 0 ! plus b
d Date DEFAULT CAST(\'2015-12-29\' AS Date)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)

View File

@ -1,25 +1,25 @@
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST((val2 + 1) AS UInt8)
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST((val2 + 1) AS UInt8)
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST((val2 + 1) AS UInt8)
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST((val2 + 1) AS UInt8)
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
1 UInt8

View File

@ -0,0 +1,12 @@
hello
hello
hello
hello
hello
hello
hello
hello
CREATE TABLE test.cast ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello

View File

@ -0,0 +1,46 @@
SELECT CAST(1 AS Enum8('hello' = 1, 'world' = 2));
SELECT cast(1 AS Enum8('hello' = 1, 'world' = 2));
SELECT CAST(1, 'Enum8(\'hello\' = 1, \'world\' = 2)');
SELECT cast(1, 'Enum8(\'hello\' = 1, \'world\' = 2)');
SELECT CAST(1 AS Enum8(
'hello' = 1,
'world' = 2));
SELECT cast(1 AS Enum8(
'hello' = 1,
'world' = 2));
SELECT CAST(1, 'Enum8(\'hello\' = 1,\n\t\'world\' = 2)');
SELECT cast(1, 'Enum8(\'hello\' = 1,\n\t\'world\' = 2)');
DROP TABLE IF EXISTS test.cast;
CREATE TABLE test.cast
(
x UInt8,
e Enum8
(
'hello' = 1,
'world' = 2
)
DEFAULT
CAST
(
x
AS
Enum8
(
'hello' = 1,
'world' = 2
)
)
) ENGINE = MergeTree ORDER BY e;
SHOW CREATE TABLE test.cast FORMAT TSVRaw;
DESC TABLE test.cast;
INSERT INTO test.cast (x) VALUES (1);
SELECT * FROM test.cast;
DROP TABLE test.cast;

View File

@ -0,0 +1,5 @@
CREATE TABLE test.cast1 ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello
1 hello

View File

@ -0,0 +1,38 @@
DROP TABLE IF EXISTS test.cast1;
DROP TABLE IF EXISTS test.cast2;
CREATE TABLE test.cast1
(
x UInt8,
e Enum8
(
'hello' = 1,
'world' = 2
)
DEFAULT
CAST
(
x
AS
Enum8
(
'hello' = 1,
'world' = 2
)
)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e;
SHOW CREATE TABLE test.cast1 FORMAT TSVRaw;
DESC TABLE test.cast1;
INSERT INTO test.cast1 (x) VALUES (1);
SELECT * FROM test.cast1;
CREATE TABLE test.cast2 AS test.cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r2') ORDER BY e;
SYSTEM SYNC REPLICA test.cast2;
SELECT * FROM test.cast2;
DROP TABLE test.cast1;
DROP TABLE test.cast2;