Introduce ASTDataType

This commit is contained in:
Alexey Milovidov 2024-07-15 02:54:29 +02:00
parent 2e1f00dd97
commit 7d9b7cc796
17 changed files with 156 additions and 76 deletions

View File

@ -2,7 +2,7 @@
#include <DataTypes/DataTypeCustom.h> #include <DataTypes/DataTypeCustom.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -83,15 +83,9 @@ DataTypePtr DataTypeFactory::tryGet(const ASTPtr & ast) const
template <bool nullptr_on_error> template <bool nullptr_on_error>
DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const
{ {
if (const auto * func = ast->as<ASTFunction>()) if (const auto * type = ast->as<ASTDataType>())
{ {
if (func->parameters) return getImpl<nullptr_on_error>(type->name, type->arguments);
{
if constexpr (nullptr_on_error)
return nullptr;
throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Data type cannot have multiple parenthesized parameters.");
}
return getImpl<nullptr_on_error>(func->name, func->arguments);
} }
if (const auto * ident = ast->as<ASTIdentifier>()) if (const auto * ident = ast->as<ASTIdentifier>())
@ -107,7 +101,7 @@ DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const
if constexpr (nullptr_on_error) if constexpr (nullptr_on_error)
return nullptr; return nullptr;
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type."); throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type: {}.", ast->getID());
} }
DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const

View File

@ -149,7 +149,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_
columns = metadata_ptr->columns.getAll(); columns = metadata_ptr->columns.getAll();
for (const auto & column_name_and_type: columns) for (const auto & column_name_and_type: columns)
{ {
const auto & ast_column_declaration = std::make_shared<ASTColumnDeclaration>(); const auto ast_column_declaration = std::make_shared<ASTColumnDeclaration>();
ast_column_declaration->name = column_name_and_type.name; ast_column_declaration->name = column_name_and_type.name;
/// parser typename /// parser typename
{ {

View File

@ -12,9 +12,9 @@
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Common/parseRemoteDescription.h> #include <Common/parseRemoteDescription.h>
#include <Databases/DatabaseFactory.h> #include <Databases/DatabaseFactory.h>
@ -25,6 +25,7 @@
#include <Core/Settings.h> #include <Core/Settings.h>
#include <filesystem> #include <filesystem>
namespace fs = std::filesystem; namespace fs = std::filesystem;
namespace DB namespace DB
@ -432,7 +433,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co
auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto metadata_snapshot = storage->getInMemoryMetadataPtr();
for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary())
{ {
const auto & column_declaration = std::make_shared<ASTColumnDeclaration>(); const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = column_type_and_name.name; column_declaration->name = column_type_and_name.name;
column_declaration->type = getColumnDeclaration(column_type_and_name.type); column_declaration->type = getColumnDeclaration(column_type_and_name.type);
columns_expression_list->children.emplace_back(column_declaration); columns_expression_list->children.emplace_back(column_declaration);
@ -470,17 +471,15 @@ ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) c
WhichDataType which(data_type); WhichDataType which(data_type);
if (which.isNullable()) if (which.isNullable())
return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType())); return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType()));
if (which.isArray()) if (which.isArray())
return makeASTFunction("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType())); return makeASTDataType("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType()));
if (which.isDateTime64()) if (which.isDateTime64())
{ return makeASTDataType("DateTime64", std::make_shared<ASTLiteral>(static_cast<UInt32>(6)));
return makeASTFunction("DateTime64", std::make_shared<ASTLiteral>(static_cast<UInt32>(6)));
}
return std::make_shared<ASTIdentifier>(data_type->getName()); return makeASTDataType(data_type->getName());
} }
void registerDatabasePostgreSQL(DatabaseFactory & factory) void registerDatabasePostgreSQL(DatabaseFactory & factory)

View File

@ -1,9 +1,7 @@
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Parsers/TablePropertiesQueriesASTs.h> #include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/formatAST.h>
#include <Processors/Sources/SourceFromSingleChunk.h> #include <Processors/Sources/SourceFromSingleChunk.h>
#include <QueryPipeline/BlockIO.h> #include <QueryPipeline/BlockIO.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>

View File

@ -3,6 +3,7 @@
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
@ -29,6 +30,7 @@
#include <Interpreters/applyTableOverride.h> #include <Interpreters/applyTableOverride.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
namespace DB namespace DB
{ {
@ -95,22 +97,22 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
} }
ASTPtr data_type = declare_column->data_type; ASTPtr data_type = declare_column->data_type;
auto * data_type_function = data_type->as<ASTFunction>(); auto * data_type_node = data_type->as<ASTDataType>();
if (data_type_function) if (data_type_node)
{ {
String type_name_upper = Poco::toUpper(data_type_function->name); String type_name_upper = Poco::toUpper(data_type_node->name);
if (is_unsigned) if (is_unsigned)
{ {
/// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED) /// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED)
if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED") if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED")
&& !endsWith(type_name_upper, "UNSIGNED")) && !endsWith(type_name_upper, "UNSIGNED"))
data_type_function->name = type_name_upper + " UNSIGNED"; data_type_node->name = type_name_upper + " UNSIGNED";
} }
if (type_name_upper == "SET") if (type_name_upper == "SET")
data_type_function->arguments.reset(); data_type_node->arguments.reset();
/// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs /// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs
/// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3) /// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3)
@ -119,7 +121,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
if (type_name_upper.find("ENUM") != String::npos) if (type_name_upper.find("ENUM") != String::npos)
{ {
UInt16 i = 0; UInt16 i = 0;
for (ASTPtr & child : data_type_function->arguments->children) for (ASTPtr & child : data_type_node->arguments->children)
{ {
auto new_child = std::make_shared<ASTFunction>(); auto new_child = std::make_shared<ASTFunction>();
new_child->name = "equals"; new_child->name = "equals";
@ -133,10 +135,10 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition)
} }
if (type_name_upper == "DATE") if (type_name_upper == "DATE")
data_type_function->name = "Date32"; data_type_node->name = "Date32";
} }
if (is_nullable) if (is_nullable)
data_type = makeASTFunction("Nullable", data_type); data_type = makeASTDataType("Nullable", data_type);
columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type)); columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type));
} }
@ -175,7 +177,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col
return columns_description; return columns_description;
} }
static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, const NamesAndTypesList & columns) static NamesAndTypesList getNames(const ASTDataType & expr, ContextPtr context, const NamesAndTypesList & columns)
{ {
if (expr.arguments->children.empty()) if (expr.arguments->children.empty())
return NamesAndTypesList{}; return NamesAndTypesList{};
@ -219,9 +221,9 @@ static std::tuple<NamesAndTypesList, NamesAndTypesList, NamesAndTypesList, NameS
ASTExpressionList * columns_definition, ASTExpressionList * indices_define, ContextPtr context, NamesAndTypesList & columns) ASTExpressionList * columns_definition, ASTExpressionList * indices_define, ContextPtr context, NamesAndTypesList & columns)
{ {
NameSet increment_columns; NameSet increment_columns;
auto keys = makeASTFunction("tuple"); auto keys = makeASTDataType("tuple");
auto unique_keys = makeASTFunction("tuple"); auto unique_keys = makeASTDataType("tuple");
auto primary_keys = makeASTFunction("tuple"); auto primary_keys = makeASTDataType("tuple");
if (indices_define && !indices_define->children.empty()) if (indices_define && !indices_define->children.empty())
{ {
@ -482,7 +484,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
{ {
auto column_declaration = std::make_shared<ASTColumnDeclaration>(); auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = name; column_declaration->name = name;
column_declaration->type = makeASTFunction(type); column_declaration->type = makeASTDataType(type);
column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_specifier = "MATERIALIZED";
column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value); column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value);
column_declaration->children.emplace_back(column_declaration->type); column_declaration->children.emplace_back(column_declaration->type);

View File

@ -1,11 +1,14 @@
#pragma once #pragma once
#include "Access/ContextAccess.h"
#include "Interpreters/Context.h" #include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Core/Settings.h> #include <Core/Settings.h>
namespace DB namespace DB
{ {
struct SecretHidingFormatSettings struct SecretHidingFormatSettings
{ {
// We can't store const Context& as there's a dangerous usage {.ctx = *getContext()} // We can't store const Context& as there's a dangerous usage {.ctx = *getContext()}
@ -24,4 +27,5 @@ inline String format(const SecretHidingFormatSettings & settings)
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets); return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
} }
} }

View File

@ -1,8 +1,6 @@
#include <Parsers/ASTColumnDeclaration.h> #include <Parsers/ASTColumnDeclaration.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB namespace DB
@ -15,8 +13,6 @@ ASTPtr ASTColumnDeclaration::clone() const
if (type) if (type)
{ {
// Type may be an ASTFunction (e.g. `create table t (a Decimal(9,0))`),
// so we have to clone it properly as well.
res->type = type->clone(); res->type = type->clone();
res->children.push_back(res->type); res->children.push_back(res->type);
} }

View File

@ -0,0 +1,57 @@
#include <Parsers/ASTDataType.h>
#include <Common/SipHash.h>
#include <IO/Operators.h>
namespace DB
{
String ASTDataType::getID(char delim) const
{
return "DataType" + (delim + name);
}
ASTPtr ASTDataType::clone() const
{
auto res = std::make_shared<ASTDataType>(*this);
res->children.clear();
if (arguments)
{
res->arguments = arguments->clone();
res->children.push_back(res->arguments);
}
return res;
}
void ASTDataType::updateTreeHashImpl(SipHash & hash_state, bool) const
{
hash_state.update(name.size());
hash_state.update(name);
/// Children are hashed automatically.
}
void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_function : "") << name;
if (arguments && !arguments->children.empty())
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
}
settings.ostr << (settings.hilite ? hilite_none : "");
}
}

36
src/Parsers/ASTDataType.h Normal file
View File

@ -0,0 +1,36 @@
#pragma once
#include <Parsers/ASTExpressionList.h>
namespace DB
{
/// AST for data types, e.g. UInt8 or Tuple(x UInt8, y Enum(a = 1))
class ASTDataType : public IAST
{
public:
String name;
ASTPtr arguments;
String getID(char delim) const override;
ASTPtr clone() const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
template <typename... Args>
std::shared_ptr<ASTDataType> makeASTDataType(const String & name, Args &&... args)
{
auto function = std::make_shared<ASTDataType>();
function->name = name;
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = { std::forward<Args>(args)... };
return function;
}
}

View File

@ -7,7 +7,6 @@
#include <Common/FieldVisitorToString.h> #include <Common/FieldVisitorToString.h>
#include <Common/KnownObjectNames.h> #include <Common/KnownObjectNames.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -19,9 +18,6 @@
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Parsers/FunctionSecretArgumentsFinderAST.h> #include <Parsers/FunctionSecretArgumentsFinderAST.h>
#include <Core/QualifiedTableName.h>
#include <boost/algorithm/string.hpp>
using namespace std::literals; using namespace std::literals;

View File

@ -46,7 +46,7 @@ public:
NullsAction nulls_action = NullsAction::EMPTY; NullsAction nulls_action = NullsAction::EMPTY;
/// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names. /// do not print empty parentheses if there are no args - compatibility with engine names.
bool no_empty_args = false; bool no_empty_args = false;
/// Specifies where this function-like expression is used. /// Specifies where this function-like expression is used.

View File

@ -271,16 +271,15 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID());
} }
// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const; String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
/* /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about
* behaviour is due to the fact such functions are called from Client which knows nothing about * access rights and settings. Moreover, the only use case for displaying secrets are backups,
* access rights and settings. Moreover, the only use case for displaying secrets are backups, * and backup tools use only direct input and ignore logs and error messages.
* and backup tools use only direct input and ignore logs and error messages. */
*/
String formatForLogging(size_t max_length = 0) const String formatForLogging(size_t max_length = 0) const
{ {
return formatWithPossiblyHidingSensitiveData(max_length, true, false); return formatWithPossiblyHidingSensitiveData(max_length, true, false);

View File

@ -5,6 +5,7 @@
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTForeignKeyDeclaration.h> #include <Parsers/ASTForeignKeyDeclaration.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIndexDeclaration.h> #include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTStatisticsDeclaration.h> #include <Parsers/ASTStatisticsDeclaration.h>
@ -75,9 +76,9 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!close.ignore(pos, expected)) if (!close.ignore(pos, expected))
return false; return false;
auto func = std::make_shared<ASTFunction>(); auto func = std::make_shared<ASTDataType>();
tryGetIdentifierNameInto(name, func->name); tryGetIdentifierNameInto(name, func->name);
// FIXME(ilezhankin): func->no_empty_args = true; ?
func->arguments = columns; func->arguments = columns;
func->children.push_back(columns); func->children.push_back(columns);
node = func; node = func;
@ -749,7 +750,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto * table_id = table->as<ASTTableIdentifier>(); auto * table_id = table->as<ASTTableIdentifier>();
// Shortcut for ATTACH a previously detached table /// A shortcut for ATTACH a previously detached table.
bool short_attach = attach && !from_path; bool short_attach = attach && !from_path;
if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon)) if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
{ {

View File

@ -101,17 +101,15 @@ class IParserColumnDeclaration : public IParserBase
{ {
public: public:
explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false)
: require_type(require_type_) : require_type(require_type_)
, allow_null_modifiers(allow_null_modifiers_) , allow_null_modifiers(allow_null_modifiers_)
, check_keywords_after_name(check_keywords_after_name_) , check_keywords_after_name(check_keywords_after_name_)
{ {
} }
void enableCheckTypeKeyword() { check_type_keyword = true; } void enableCheckTypeKeyword() { check_type_keyword = true; }
protected: protected:
using ASTDeclarePtr = std::shared_ptr<ASTColumnDeclaration>;
const char * getName() const override{ return "column declaration"; } const char * getName() const override{ return "column declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;

View File

@ -1,6 +1,7 @@
#include <Parsers/ParserDataType.h> #include <Parsers/ParserDataType.h>
#include <boost/algorithm/string/case_conv.hpp> #include <boost/algorithm/string/case_conv.hpp>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIdentifier_fwd.h> #include <Parsers/ASTIdentifier_fwd.h>
@ -198,13 +199,12 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
} }
} }
auto function_node = std::make_shared<ASTFunction>(); auto data_type_node = std::make_shared<ASTDataType>();
function_node->name = type_name; data_type_node->name = type_name;
function_node->no_empty_args = true;
if (pos->type != TokenType::OpeningRoundBracket) if (pos->type != TokenType::OpeningRoundBracket)
{ {
node = function_node; node = data_type_node;
return true; return true;
} }
++pos; ++pos;
@ -222,10 +222,10 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
++pos; ++pos;
function_node->arguments = expr_list_args; data_type_node->arguments = expr_list_args;
function_node->children.push_back(function_node->arguments); data_type_node->children.push_back(data_type_node->arguments);
node = function_node; node = data_type_node;
return true; return true;
} }

View File

@ -7,10 +7,8 @@
#include <Common/parseAddress.h> #include <Common/parseAddress.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <Core/Settings.h>
#include <Core/PostgreSQL/Connection.h> #include <Core/PostgreSQL/Connection.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesDecimal.h> #include <DataTypes/DataTypesDecimal.h>
@ -22,6 +20,7 @@
#include <Processors/QueryPlan/QueryPlan.h> #include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h> #include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
@ -295,7 +294,7 @@ std::shared_ptr<ASTColumnDeclaration> StorageMaterializedPostgreSQL::getMaterial
auto column_declaration = std::make_shared<ASTColumnDeclaration>(); auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = std::move(name); column_declaration->name = std::move(name);
column_declaration->type = makeASTFunction(type); column_declaration->type = makeASTDataType(type);
column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_specifier = "MATERIALIZED";
column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value); column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value);
@ -312,17 +311,17 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d
WhichDataType which(data_type); WhichDataType which(data_type);
if (which.isNullable()) if (which.isNullable())
return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType())); return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType()));
if (which.isArray()) if (which.isArray())
return makeASTFunction("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType())); return makeASTDataType("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType()));
/// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it /// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it
if (which.isDecimal()) if (which.isDecimal())
{ {
auto make_decimal_expression = [&](std::string type_name) auto make_decimal_expression = [&](std::string type_name)
{ {
auto ast_expression = std::make_shared<ASTFunction>(); auto ast_expression = std::make_shared<ASTDataType>();
ast_expression->name = type_name; ast_expression->name = type_name;
ast_expression->arguments = std::make_shared<ASTExpressionList>(); ast_expression->arguments = std::make_shared<ASTExpressionList>();
@ -354,7 +353,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d
return ast_expression; return ast_expression;
} }
return std::make_shared<ASTIdentifier>(data_type->getName()); return makeASTDataType(data_type->getName());
} }

View File

@ -20,6 +20,7 @@
#include <Parsers/ASTAsterisk.h> #include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTDropQuery.h> #include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h> #include <Parsers/ASTOrderByElement.h>
@ -805,7 +806,7 @@ ASTPtr StorageWindowView::getInnerTableCreateQuery(const ASTPtr & inner_query, c
{ {
auto column_window = std::make_shared<ASTColumnDeclaration>(); auto column_window = std::make_shared<ASTColumnDeclaration>();
column_window->name = window_id_name; column_window->name = window_id_name;
column_window->type = std::make_shared<ASTIdentifier>("UInt32"); column_window->type = makeASTDataType("UInt32");
columns_list->children.push_back(column_window); columns_list->children.push_back(column_window);
} }