提交 6d6bad51 编写于 作者: A alesapin 提交者: alexey-milovidov

CLICKHOUSE-3772: Fixes in system tables formats, data_type_families, aggregate_function_combinators

上级 3e5daae2
......@@ -59,13 +59,19 @@ DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & para
return it->second(parameters);
}
String family_name_lowercase = Poco::toLower(family_name);
{
String family_name_lowercase = Poco::toLower(family_name);
DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase);
if (case_insensitive_data_types.end() != it)
return it->second(parameters);
}
if (auto it = aliases.find(family_name); it != aliases.end())
return get(it->second, parameters);
else if (auto it = case_insensitive_aliases.find(family_name_lowercase); it != case_insensitive_aliases.end())
return get(it->second, parameters);
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
}
......@@ -76,11 +82,16 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat
throw Exception("DataTypeFactory: the data type family " + family_name + " has been provided "
" a null constructor", ErrorCodes::LOGICAL_ERROR);
String family_name_lowercase = Poco::toLower(family_name);
if (aliases.count(family_name) || case_insensitive_aliases.count(family_name_lowercase))
throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is already registered as alias",
ErrorCodes::LOGICAL_ERROR);
if (!data_types.emplace(family_name, creator).second)
throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
String family_name_lowercase = Poco::toLower(family_name);
if (case_sensitiveness == CaseInsensitive
&& !case_insensitive_data_types.emplace(family_name_lowercase, creator).second)
......@@ -88,6 +99,27 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat
ErrorCodes::LOGICAL_ERROR);
}
void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) {
String real_type_dict_name;
if (data_types.count(real_name))
real_type_dict_name = real_name;
else if (auto type_name_lowercase = Poco::toLower(real_name); case_insensitive_data_types.count(type_name_lowercase))
real_type_dict_name = type_name_lowercase;
else
throw Exception("DataTypeFactory: can't create alias '" + alias_name + "' the data type family '" + real_name + "' is not registered", ErrorCodes::LOGICAL_ERROR);
String alias_name_lowercase = Poco::toLower(alias_name);
if (data_types.count(alias_name) || case_insensitive_data_types.count(alias_name_lowercase))
throw Exception("DataTypeFactory: the alias name " + alias_name + " is already registered as datatype", ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive)
if(!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second)
throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
if(!aliases.emplace(alias_name, real_type_dict_name).second)
throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
{
......@@ -103,6 +135,35 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
}, case_sensitiveness);
}
std::vector<String> DataTypeFactory::getAllDataTypeNames() const
{
std::vector<String> result;
auto getter = [] (const auto& pair) { return pair.first; };
std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter);
std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter);
return result;
}
bool DataTypeFactory::isCaseInsensitive(const String & name) const
{
String name_lowercase = Poco::toLower(name);
return case_insensitive_data_types.count(name_lowercase) || case_insensitive_aliases.count(name_lowercase);
}
const String & DataTypeFactory::aliasTo(const String & name) const
{
if (auto it = aliases.find(name); it != aliases.end())
return it->second;
else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end())
return it->second;
throw Exception("DataTypeFactory: the data type '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR);
}
bool DataTypeFactory::isAlias(const String & name) const {
return aliases.count(name) || case_insensitive_aliases.count(name);
}
void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory);
......
......@@ -25,6 +25,7 @@ private:
using Creator = std::function<DataTypePtr(const ASTPtr & parameters)>;
using SimpleCreator = std::function<DataTypePtr()>;
using DataTypesDictionary = std::unordered_map<String, Creator>;
using AliasMap = std::unordered_map<String, String>; // alias -> original type
public:
DataTypePtr get(const String & full_name) const;
......@@ -44,10 +45,18 @@ public:
/// Register a simple data type, that have no parameters.
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
const DataTypesDictionary & getAllDataTypes() const
{
return data_types;
}
/** Register additional name for datatype.
* real_name datatype have to be already registered.
*/
void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive);
std::vector<String> getAllDataTypeNames() const;
bool isCaseInsensitive(const String & name) const;
const String & aliasTo(const String & name) const;
bool isAlias(const String & name) const;
private:
DataTypesDictionary data_types;
......@@ -55,6 +64,12 @@ private:
/// Case insensitive data types will be additionally added here with lowercased name.
DataTypesDictionary case_insensitive_data_types;
/// Alias map to data_types from previous two maps
AliasMap aliases;
/// Case insensitive aliases
AliasMap case_insensitive_aliases;
DataTypeFactory();
friend class ext::singleton<DataTypeFactory>;
};
......
......@@ -231,7 +231,7 @@ void registerDataTypeFixedString(DataTypeFactory & factory)
factory.registerDataType("FixedString", create);
/// Compatibility alias.
factory.registerDataType("BINARY", create, DataTypeFactory::CaseInsensitive);
factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive);
}
}
......@@ -312,16 +312,16 @@ void registerDataTypeString(DataTypeFactory & factory)
/// These synonims are added for compatibility.
factory.registerSimpleDataType("CHAR", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("VARCHAR", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TINYTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("MEDIUMTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("LONGTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("BLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TINYBLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("MEDIUMBLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("LONGBLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive);
factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive);
}
}
......@@ -22,13 +22,13 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
/// These synonims are added for compatibility.
factory.registerSimpleDataType("TINYINT", [] { return DataTypePtr(std::make_shared<DataTypeInt8>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("SMALLINT", [] { return DataTypePtr(std::make_shared<DataTypeInt16>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("INT", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("INTEGER", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("BIGINT", [] { return DataTypePtr(std::make_shared<DataTypeInt64>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("FLOAT", [] { return DataTypePtr(std::make_shared<DataTypeFloat32>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("DOUBLE", [] { return DataTypePtr(std::make_shared<DataTypeFloat64>()); }, DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive);
}
}
......@@ -14,21 +14,15 @@ class Context;
/** Base class for system tables whose all columns have String type.
*/
template <typename Self>
class IStorageSystemWithStringColumns : public IStorage
class IStorageSystemOneBlock : public IStorage
{
protected:
virtual void fillData(MutableColumns & res_columns) const = 0;
public:
IStorageSystemWithStringColumns(const String & name_) : name(name_)
IStorageSystemOneBlock(const String & name_) : name(name_)
{
auto names = Self::getColumnNames();
NamesAndTypesList name_list;
for (const auto & name : names)
{
name_list.push_back(NameAndTypePair{name, std::make_shared<DataTypeString>()});
}
setColumns(ColumnsDescription(name_list));
setColumns(ColumnsDescription(Self::getNamesAndTypes()));
}
std::string getTableName() const override
......
......@@ -9,6 +9,10 @@ void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_co
for (const auto & pair : combinators)
{
res_columns[0]->insert(pair.first);
if (pair.first != "Null")
res_columns[1]->insert(UInt64(0));
else
res_columns[1]->insert(UInt64(1));
}
}
}
#pragma once
#include <Storages/System/IStorageSystemWithStringColumns.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>,
public IStorageSystemWithStringColumns<StorageSystemAggregateFunctionCombinators>
public IStorageSystemOneBlock<StorageSystemAggregateFunctionCombinators>
{
protected:
void fillData(MutableColumns & res_columns) const override;
public:
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
std::string getName() const override
{
return "SystemAggregateFunctionCombinators";
}
static std::vector<String> getColumnNames()
static NamesAndTypesList getNamesAndTypes()
{
return {"name"};
return {
{"name", std::make_shared<DataTypeString>()},
{"is_internal", std::make_shared<DataTypeUInt8>()},
};
}
};
}
......@@ -5,9 +5,9 @@ namespace DB
{
void StorageSystemCollations::fillData(MutableColumns & res_columns) const
{
for (const auto & collation : Collator::getAvailableCollations())
for (const auto & collation_name : Collator::getAvailableCollations())
{
res_columns[0]->insert(collation);
res_columns[0]->insert(collation_name);
}
}
}
#pragma once
#include <Storages/System/IStorageSystemWithStringColumns.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemCollations : public ext::shared_ptr_helper<StorageSystemCollations>,
public IStorageSystemWithStringColumns<StorageSystemCollations>
public IStorageSystemOneBlock<StorageSystemCollations>
{
protected:
void fillData(MutableColumns & res_columns) const override;
public:
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
std::string getName() const override
{
return "SystemTableCollations";
}
static std::vector<String> getColumnNames()
static NamesAndTypesList getNamesAndTypes()
{
return {"name"};
return {
{"name", std::make_shared<DataTypeString>()},
};
}
};
}
......@@ -12,39 +12,37 @@ namespace DB
{
namespace
{
String getPropertiesAsString(const DataTypePtr data_type)
void setTypePropertries(const DataTypePtr data_type, MutableColumns & res_columns)
{
std::vector<std::string> properties;
if (data_type->isParametric())
properties.push_back("parametric");
if (data_type->haveSubtypes())
properties.push_back("have_subtypes");
if (data_type->cannotBeStoredInTables())
properties.push_back("cannot_be_stored_in_tables");
if (data_type->isComparable())
properties.push_back("comparable");
if (data_type->canBeComparedWithCollation())
properties.push_back("can_be_compared_with_collation");
if (data_type->canBeUsedAsVersion())
properties.push_back("can_be_used_as_version");
if (data_type->isSummable())
properties.push_back("summable");
if (data_type->canBeUsedInBitOperations())
properties.push_back("can_be_used_in_bit_operations");
if (data_type->canBeUsedInBooleanContext())
properties.push_back("can_be_used_in_boolean_context");
if (data_type->isValueRepresentedByNumber())
properties.push_back("value_represented_by_number");
if (data_type->isCategorial())
properties.push_back("categorial");
if (data_type->isNullable())
properties.push_back("nullable");
if (data_type->onlyNull())
properties.push_back("only_null");
if (data_type->canBeInsideNullable())
properties.push_back("can_be_inside_nullable");
return boost::algorithm::join(properties, ",");
res_columns[3]->insert(UInt64(data_type->isParametric()));
res_columns[4]->insert(UInt64(data_type->haveSubtypes()));
res_columns[5]->insert(UInt64(data_type->cannotBeStoredInTables()));
res_columns[6]->insert(UInt64(data_type->isComparable()));
res_columns[7]->insert(UInt64(data_type->canBeComparedWithCollation()));
res_columns[8]->insert(UInt64(data_type->canBeUsedAsVersion()));
res_columns[9]->insert(UInt64(data_type->isSummable()));
res_columns[10]->insert(UInt64(data_type->canBeUsedInBitOperations()));
res_columns[11]->insert(UInt64(data_type->canBeUsedInBooleanContext()));
res_columns[12]->insert(UInt64(data_type->isCategorial()));
res_columns[13]->insert(UInt64(data_type->isNullable()));
res_columns[14]->insert(UInt64(data_type->onlyNull()));
res_columns[15]->insert(UInt64(data_type->canBeInsideNullable()));
}
void setComplexTypeProperties(const String & name, MutableColumns & res_columns)
{
res_columns[3]->insert(UInt64(1)); //complex types are always parametric
if (name == "AggregateFunction")
res_columns[4]->insert(UInt64(0));
else if (name == "Tuple")
res_columns[4]->insert(Null());
else
res_columns[4]->insert(UInt64(1));
for (size_t i = 5; i < StorageSystemDataTypeFamilies::getNamesAndTypes().size(); ++i)
res_columns[i]->insert(Null());
}
ASTPtr createFakeEnumCreationAst()
{
String fakename{"e"};
......@@ -56,35 +54,46 @@ namespace
clone->children.push_back(ast_func);
return clone;
}
ASTPtr createFakeFixedStringAst()
{
ASTPtr result = std::make_shared<ASTLiteral>(Field(UInt64(1)));
auto clone = result->clone();
clone->children.clear();
clone->children.push_back(result);
return clone;
}
}
void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const
{
const auto & factory = DataTypeFactory::instance();
const auto & data_types = factory.getAllDataTypes();
for (const auto & pair : data_types)
auto names = factory.getAllDataTypeNames();
for (const auto & name : names)
{
res_columns[0]->insert(pair.first);
res_columns[0]->insert(name);
res_columns[1]->insert(UInt64(factory.isCaseInsensitive(name)));
if (factory.isAlias(name))
res_columns[2]->insert(factory.aliasTo(name));
else
res_columns[2]->insert(String(""));
try
{
DataTypePtr type_ptr;
//special case with enum, because it has arguments but it's properties doesn't
//depend on arguments
if (boost::starts_with(pair.first, "Enum"))
{
type_ptr = factory.get(pair.first, createFakeEnumCreationAst());
}
// hardcoded cases for simple parametric types
if (boost::starts_with(name, "Enum"))
type_ptr = factory.get(name, createFakeEnumCreationAst());
else if (name == "FixedString" || name == "BINARY")
type_ptr = factory.get(name, createFakeFixedStringAst());
else
{
type_ptr = factory.get(pair.first);
}
res_columns[1]->insert(getPropertiesAsString(type_ptr));
type_ptr = factory.get(name);
setTypePropertries(type_ptr, res_columns);
}
catch (Exception & ex)
{
res_columns[1]->insert(String{"depends_on_arguments"});
setComplexTypeProperties(name, res_columns);
}
}
}
......
#pragma once
#include <Storages/System/IStorageSystemWithStringColumns.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper<StorageSystemDataTypeFamilies>,
public IStorageSystemWithStringColumns<StorageSystemDataTypeFamilies>
public IStorageSystemOneBlock<StorageSystemDataTypeFamilies>
{
protected:
void fillData(MutableColumns & res_columns) const override;
public:
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
std::string getName() const override
{
return "SystemTableDataTypeFamilies";
}
static std::vector<String> getColumnNames()
static NamesAndTypesList getNamesAndTypes()
{
return {"name", "properties"};
return {
{"name", std::make_shared<DataTypeString>()},
{"case_insensivie", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"alias_to", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"parametric", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"have_subtypes", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"cannot_be_stored_in_tables", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"comparable", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"can_be_compared_with_collation", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"can_be_used_as_version", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"is_summable", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"can_be_used_in_bit_operations", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"can_be_used_in_boolean_context", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"categorial", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"nullable", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"only_null", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"can_be_inside_nullable", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
};
}
};
}
......@@ -9,22 +9,11 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns) const
for (const auto & pair : formats)
{
const auto & [name, creator_pair] = pair;
bool has_input_format = (creator_pair.first != nullptr);
bool has_output_format = (creator_pair.second != nullptr);
UInt64 has_input_format = UInt64(creator_pair.first != nullptr);
UInt64 has_output_format = UInt64(creator_pair.second != nullptr);
res_columns[0]->insert(name);
std::string format_type;
if (has_input_format)
format_type = "input";
if (has_output_format)
{
if (!format_type.empty())
format_type += "/output";
else
format_type = "output";
}
res_columns[1]->insert(format_type);
res_columns[1]->insert(has_input_format);
res_columns[2]->insert(has_output_format);
}
}
}
#pragma once
#include <Storages/System/IStorageSystemWithStringColumns.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemFormats : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemWithStringColumns<StorageSystemFormats>
class StorageSystemFormats : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemOneBlock<StorageSystemFormats>
{
protected:
void fillData(MutableColumns & res_columns) const override;
public:
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
std::string getName() const override
{
return "SystemFormats";
}
static std::vector<String> getColumnNames()
static NamesAndTypesList getNamesAndTypes()
{
return {"name", "type"};
return {
{"name", std::make_shared<DataTypeString>()},
{"is_input", std::make_shared<DataTypeUInt8>()},
{"is_output", std::make_shared<DataTypeUInt8>()},
};
}
};
}
#pragma once
#include <Storages/System/IStorageSystemWithStringColumns.h>
#include <DataTypes/DataTypeString.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemTableFunctions : public ext::shared_ptr_helper<StorageSystemTableFunctions>,
public IStorageSystemWithStringColumns<StorageSystemTableFunctions>
public IStorageSystemOneBlock<StorageSystemTableFunctions>
{
protected:
void fillData(MutableColumns & res_columns) const override;
public:
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
std::string getName() const override
{
return "SystemTableFunctions";
}
static std::vector<String> getColumnNames()
static NamesAndTypesList getNamesAndTypes()
{
return {"name"};
return {{"name", std::make_shared<DataTypeString>()}};
}
};
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册