Make use_mysql_types_in_show_columns affect only SHOW COLUMNS

This commit is contained in:
Robert Schulze 2023-10-10 21:44:08 +00:00
parent 8b7d1021cd
commit bd43b84bf8
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
29 changed files with 54 additions and 108 deletions

View File

@ -45,7 +45,6 @@ public:
String doGetName() const override; String doGetName() const override;
String getNameWithoutVersion() const; String getNameWithoutVersion() const;
const char * getFamilyName() const override { return "AggregateFunction"; } const char * getFamilyName() const override { return "AggregateFunction"; }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; }
Array getParameters() const { return parameters; } Array getParameters() const { return parameters; }

View File

@ -35,10 +35,6 @@ public:
{ {
return "Array"; return "Array";
} }
String getSQLCompatibleName() const override
{
return "TEXT";
}
bool canBeInsideNullable() const override bool canBeInsideNullable() const override
{ {

View File

@ -13,7 +13,6 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::Date; } TypeIndex getTypeId() const override { return TypeIndex::Date; }
const char * getFamilyName() const override { return family_name; } const char * getFamilyName() const override { return family_name; }
String getSQLCompatibleName() const override { return "DATE"; }
bool canBeUsedAsVersion() const override { return true; } bool canBeUsedAsVersion() const override { return true; }
bool canBeInsideNullable() const override { return true; } bool canBeInsideNullable() const override { return true; }

View File

@ -13,7 +13,6 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::Date32; } TypeIndex getTypeId() const override { return TypeIndex::Date32; }
const char * getFamilyName() const override { return family_name; } const char * getFamilyName() const override { return family_name; }
String getSQLCompatibleName() const override { return "DATE"; }
Field getDefault() const override Field getDefault() const override
{ {

View File

@ -38,7 +38,6 @@ public:
static constexpr auto family_name = "DateTime"; static constexpr auto family_name = "DateTime";
const char * getFamilyName() const override { return family_name; } const char * getFamilyName() const override { return family_name; }
String getSQLCompatibleName() const override { return "DATETIME"; }
String doGetName() const override; String doGetName() const override;
TypeIndex getTypeId() const override { return TypeIndex::DateTime; } TypeIndex getTypeId() const override { return TypeIndex::DateTime; }

View File

@ -28,7 +28,6 @@ public:
DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info);
const char * getFamilyName() const override { return family_name; } const char * getFamilyName() const override { return family_name; }
String getSQLCompatibleName() const override { return "DATETIME"; }
std::string doGetName() const override; std::string doGetName() const override;
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }

View File

@ -36,30 +36,6 @@ const char * DataTypeEnum<Type>::getFamilyName() const
return EnumName<FieldType>::value; return EnumName<FieldType>::value;
} }
template <typename Type>
std::string DataTypeEnum<Type>::generateMySQLName(const Values & values)
{
WriteBufferFromOwnString out;
writeString("ENUM", out);
writeChar('(', out);
auto first = true;
for (const auto & name_and_value : values)
{
if (!first)
writeString(", ", out);
first = false;
writeQuotedString(name_and_value.first, out);
}
writeChar(')', out);
return out.str();
}
template <typename Type> template <typename Type>
std::string DataTypeEnum<Type>::generateName(const Values & values) std::string DataTypeEnum<Type>::generateName(const Values & values)
{ {

View File

@ -46,14 +46,12 @@ public:
private: private:
std::string type_name; std::string type_name;
static std::string generateName(const Values & values); static std::string generateName(const Values & values);
static std::string generateMySQLName(const Values & values);
public: public:
explicit DataTypeEnum(const Values & values_); explicit DataTypeEnum(const Values & values_);
std::string doGetName() const override { return type_name; } std::string doGetName() const override { return type_name; }
const char * getFamilyName() const override; const char * getFamilyName() const override;
String getSQLCompatibleName() const override { return generateMySQLName(this->getValues()); }
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }

View File

@ -42,8 +42,6 @@ public:
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }
const char * getFamilyName() const override { return "FixedString"; } const char * getFamilyName() const override { return "FixedString"; }
/// Use TEXT for compatibility with MySQL to allow arbitrary bytes.
String getSQLCompatibleName() const override { return "TEXT"; }
size_t getN() const size_t getN() const
{ {

View File

@ -24,7 +24,6 @@ public:
std::string doGetName() const override; std::string doGetName() const override;
const char * getFamilyName() const override { return "Function"; } const char * getFamilyName() const override { return "Function"; }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return TypeIndex::Function; } TypeIndex getTypeId() const override { return TypeIndex::Function; }
const DataTypes & getArgumentTypes() const const DataTypes & getArgumentTypes() const

View File

@ -19,7 +19,6 @@ public:
static constexpr auto type_id = TypeToTypeIndex<IPv4>; static constexpr auto type_id = TypeToTypeIndex<IPv4>;
const char * getFamilyName() const override { return TypeName<IPv4>.data(); } const char * getFamilyName() const override { return TypeName<IPv4>.data(); }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }
@ -61,7 +60,6 @@ public:
static constexpr auto type_id = TypeToTypeIndex<IPv6>; static constexpr auto type_id = TypeToTypeIndex<IPv6>;
const char * getFamilyName() const override { return TypeName<IPv6>.data(); } const char * getFamilyName() const override { return TypeName<IPv6>.data(); }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }

View File

@ -27,7 +27,6 @@ public:
SerializationPtr doGetDefaultSerialization() const override; SerializationPtr doGetDefaultSerialization() const override;
std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); }
const char * getFamilyName() const override { return "Interval"; } const char * getFamilyName() const override { return "Interval"; }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return TypeIndex::Interval; } TypeIndex getTypeId() const override { return TypeIndex::Interval; }
bool equals(const IDataType & rhs) const override; bool equals(const IDataType & rhs) const override;

View File

@ -23,7 +23,6 @@ public:
return "LowCardinality(" + dictionary_type->getName() + ")"; return "LowCardinality(" + dictionary_type->getName() + ")";
} }
const char * getFamilyName() const override { return "LowCardinality"; } const char * getFamilyName() const override { return "LowCardinality"; }
String getSQLCompatibleName() const override { return dictionary_type->getSQLCompatibleName(); }
TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; }

View File

@ -31,7 +31,6 @@ public:
std::string doGetName() const override; std::string doGetName() const override;
std::string doGetPrettyName(size_t indent) const override; std::string doGetPrettyName(size_t indent) const override;
const char * getFamilyName() const override { return "Map"; } const char * getFamilyName() const override { return "Map"; }
String getSQLCompatibleName() const override { return "JSON"; }
bool canBeInsideNullable() const override { return false; } bool canBeInsideNullable() const override { return false; }

View File

@ -16,7 +16,6 @@ public:
static constexpr bool is_parametric = false; static constexpr bool is_parametric = false;
const char * getFamilyName() const override { return "Nothing"; } const char * getFamilyName() const override { return "Nothing"; }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return TypeIndex::Nothing; } TypeIndex getTypeId() const override { return TypeIndex::Nothing; }

View File

@ -16,7 +16,6 @@ public:
explicit DataTypeNullable(const DataTypePtr & nested_data_type_); explicit DataTypeNullable(const DataTypePtr & nested_data_type_);
std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
const char * getFamilyName() const override { return "Nullable"; } const char * getFamilyName() const override { return "Nullable"; }
String getSQLCompatibleName() const override { return nested_data_type->getSQLCompatibleName(); }
TypeIndex getTypeId() const override { return TypeIndex::Nullable; } TypeIndex getTypeId() const override { return TypeIndex::Nullable; }
MutableColumnPtr createColumn() const override; MutableColumnPtr createColumn() const override;

View File

@ -11,34 +11,6 @@ Field DataTypeNumberBase<T>::getDefault() const
{ {
return NearestFieldType<FieldType>(); return NearestFieldType<FieldType>();
} }
template <typename T>
String DataTypeNumberBase<T>::getSQLCompatibleName() const
{
if constexpr (std::is_same_v<T, Int8>)
return "TINYINT";
else if constexpr (std::is_same_v<T, Int16>)
return "SMALLINT";
else if constexpr (std::is_same_v<T, Int32>)
return "INTEGER";
else if constexpr (std::is_same_v<T, Int64>)
return "BIGINT";
else if constexpr (std::is_same_v<T, UInt8>)
return "TINYINT UNSIGNED";
else if constexpr (std::is_same_v<T, UInt16>)
return "SMALLINT UNSIGNED";
else if constexpr (std::is_same_v<T, UInt32>)
return "INTEGER UNSIGNED";
else if constexpr (std::is_same_v<T, UInt64>)
return "BIGINT UNSIGNED";
else if constexpr (std::is_same_v<T, Float32>)
return "FLOAT";
else if constexpr (std::is_same_v<T, Float64>)
return "DOUBLE";
/// Unsupported types are converted to TEXT
else
return "TEXT";
}
template <typename T> template <typename T>
MutableColumnPtr DataTypeNumberBase<T>::createColumn() const MutableColumnPtr DataTypeNumberBase<T>::createColumn() const
{ {

View File

@ -25,7 +25,6 @@ public:
using ColumnType = ColumnVector<T>; using ColumnType = ColumnVector<T>;
const char * getFamilyName() const override { return TypeName<T>.data(); } const char * getFamilyName() const override { return TypeName<T>.data(); }
String getSQLCompatibleName() const override;
TypeIndex getTypeId() const override { return TypeToTypeIndex<T>; } TypeIndex getTypeId() const override { return TypeToTypeIndex<T>; }
Field getDefault() const override; Field getDefault() const override;

View File

@ -23,7 +23,6 @@ public:
DataTypeObject(const String & schema_format_, bool is_nullable_); DataTypeObject(const String & schema_format_, bool is_nullable_);
const char * getFamilyName() const override { return "Object"; } const char * getFamilyName() const override { return "Object"; }
String getSQLCompatibleName() const override { return "JSON"; }
String doGetName() const override; String doGetName() const override;
TypeIndex getTypeId() const override { return TypeIndex::Object; } TypeIndex getTypeId() const override { return TypeIndex::Object; }

View File

@ -15,7 +15,6 @@ class DataTypeSet final : public IDataTypeDummy
public: public:
static constexpr bool is_parametric = true; static constexpr bool is_parametric = true;
const char * getFamilyName() const override { return "Set"; } const char * getFamilyName() const override { return "Set"; }
String getSQLCompatibleName() const override { return "TEXT"; }
TypeIndex getTypeId() const override { return TypeIndex::Set; } TypeIndex getTypeId() const override { return TypeIndex::Set; }
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }

View File

@ -21,8 +21,6 @@ public:
return "String"; return "String";
} }
String getSQLCompatibleName() const override { return "BLOB"; }
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }
MutableColumnPtr createColumn() const override; MutableColumnPtr createColumn() const override;

View File

@ -34,7 +34,6 @@ public:
std::string doGetName() const override; std::string doGetName() const override;
std::string doGetPrettyName(size_t indent) const override; std::string doGetPrettyName(size_t indent) const override;
const char * getFamilyName() const override { return "Tuple"; } const char * getFamilyName() const override { return "Tuple"; }
String getSQLCompatibleName() const override { return "JSON"; }
bool canBeInsideNullable() const override { return false; } bool canBeInsideNullable() const override { return false; }
bool supportsSparseSerialization() const override { return true; } bool supportsSparseSerialization() const override { return true; }

View File

@ -18,7 +18,6 @@ public:
static constexpr auto type_id = TypeIndex::UUID; static constexpr auto type_id = TypeIndex::UUID;
const char * getFamilyName() const override { return "UUID"; } const char * getFamilyName() const override { return "UUID"; }
String getSQLCompatibleName() const override { return "CHAR"; }
TypeIndex getTypeId() const override { return type_id; } TypeIndex getTypeId() const override { return type_id; }

View File

@ -28,18 +28,6 @@ std::string DataTypeDecimal<T>::doGetName() const
return fmt::format("Decimal({}, {})", this->precision, this->scale); return fmt::format("Decimal({}, {})", this->precision, this->scale);
} }
template <is_decimal T>
std::string DataTypeDecimal<T>::getSQLCompatibleName() const
{
/// See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html
/// DECIMAL(M,D)
/// M is the maximum number of digits (the precision). It has a range of 1 to 65.
/// D is the number of digits to the right of the decimal point (the scale). It has a range of 0 to 30 and must be no larger than M.
if (this->precision > 65 || this->scale > 30)
return "TEXT";
return fmt::format("DECIMAL({}, {})", this->precision, this->scale);
}
template <is_decimal T> template <is_decimal T>
bool DataTypeDecimal<T>::equals(const IDataType & rhs) const bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
{ {

View File

@ -39,7 +39,6 @@ public:
static constexpr auto family_name = "Decimal"; static constexpr auto family_name = "Decimal";
const char * getFamilyName() const override { return family_name; } const char * getFamilyName() const override { return family_name; }
String getSQLCompatibleName() const override;
std::string doGetName() const override; std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeToTypeIndex<T>; } TypeIndex getTypeId() const override { return TypeToTypeIndex<T>; }

View File

@ -83,8 +83,6 @@ public:
/// Name of data type family (example: FixedString, Array). /// Name of data type family (example: FixedString, Array).
virtual const char * getFamilyName() const = 0; virtual const char * getFamilyName() const = 0;
/// Name of corresponding data type in MySQL (exampe: Bigint, Blob, etc)
virtual String getSQLCompatibleName() const = 0;
/// Data type id. It's used for runtime type checks. /// Data type id. It's used for runtime type checks.
virtual TypeIndex getTypeId() const = 0; virtual TypeIndex getTypeId() const = 0;

View File

@ -83,11 +83,6 @@ namespace
String doGetName() const override { return finalize()->getName(); } String doGetName() const override { return finalize()->getName(); }
TypeIndex getTypeId() const override { return TypeIndex::JSONPaths; } TypeIndex getTypeId() const override { return TypeIndex::JSONPaths; }
String getSQLCompatibleName() const override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getSQLCompatibleName is not implemented for JSONObjectForInference type");
}
bool isParametric() const override bool isParametric() const override
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method isParametric is not implemented for JSONObjectForInference type"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method isParametric is not implemented for JSONObjectForInference type");

View File

@ -24,6 +24,8 @@ String InterpreterShowColumnsQuery::getRewrittenQuery()
{ {
const auto & query = query_ptr->as<ASTShowColumnsQuery &>(); const auto & query = query_ptr->as<ASTShowColumnsQuery &>();
[[maybe_unused]] const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns;
WriteBufferFromOwnString buf_database; WriteBufferFromOwnString buf_database;
String resolved_database = getContext()->resolveDatabase(query.database); String resolved_database = getContext()->resolveDatabase(query.database);
writeEscapedString(resolved_database, buf_database); writeEscapedString(resolved_database, buf_database);
@ -33,18 +35,64 @@ String InterpreterShowColumnsQuery::getRewrittenQuery()
writeEscapedString(query.table, buf_table); writeEscapedString(query.table, buf_table);
String table = buf_table.str(); String table = buf_table.str();
String rewritten_query = R"( String rewritten_query;
if (use_mysql_types)
/// Cheapskate mapping from native to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html
/// Known issues:
/// - Enums are translated to TEXT
rewritten_query += R"(
WITH map(
'Int8', 'TINYINT',
'Int16', 'SMALLINT',
'Int32', 'INTEGER',
'Int64', 'BIGINT',
'UInt8', 'TINYINT UNSIGNED',
'UInt16', 'SMALLINT UNSIGNED',
'UInt32', 'INTEGER UNSIGNED',
'UInt64', 'BIGINT UNSIGNED',
'Float32', 'FLOAT',
'Float64', 'DOUBLE',
'String', 'BLOB',
'UUID', 'CHAR',
'Bool', 'TINYINT',
'Date', 'DATE',
'Date32', 'DATE',
'DateTime', 'DATETIME',
'DateTime64', 'DATETIME',
'Map', 'JSON',
'Tuple', 'JSON',
'Object', 'JSON') AS native_to_mysql_mapping,
splitByRegexp('\(|\)', type) AS split,
multiIf(startsWith(type, 'LowCardinality(Nullable'), split[3],
startsWith(type, 'LowCardinality'), split[2],
startsWith(type, 'Nullable'), split[2],
split[1]) AS inner_type,
if(length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision,
multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'),
mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type],
'TEXT') AS mysql_type
)";
rewritten_query += R"(
SELECT SELECT
name AS field, name AS field,
)";
if (use_mysql_types)
rewritten_query += R"(
mysql_type AS type,
)";
else
rewritten_query += R"(
type AS type, type AS type,
if (startsWith(type, 'Nullable'), 'YES', 'NO') AS `null`, )";
rewritten_query += R"(
multiIf(startsWith(type, 'Nullable('), 'YES', startsWith(type, 'LowCardinality(Nullable('), 'YES', 'NO') AS `null`,
trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key,
if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default,
'' AS extra )"; '' AS extra )";
// Known issue: Field 'null' is wrong for types like 'LowCardinality(Nullable(String))'. Can't simply replace 'startsWith' by
// `hasSubsequence` as that would return `true` for non-nullable types such as `Tuple(Nullable(String), String)`...
// TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see
// IStorage::getVirtuals(). We can't easily do that via SQL. // IStorage::getVirtuals(). We can't easily do that via SQL.

View File

@ -76,7 +76,6 @@ public:
, columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_)
, databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_))
, client_info_interface(context->getClientInfo().interface) , client_info_interface(context->getClientInfo().interface)
, use_mysql_types(context->getSettingsRef().use_mysql_types_in_show_columns)
, total_tables(tables->size()), access(context->getAccess()) , total_tables(tables->size()), access(context->getAccess())
, query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout)
{ {
@ -149,7 +148,7 @@ protected:
if (columns_mask[src_index++]) if (columns_mask[src_index++])
res_columns[res_index++]->insert(column.name); res_columns[res_index++]->insert(column.name);
if (columns_mask[src_index++]) if (columns_mask[src_index++])
res_columns[res_index++]->insert(use_mysql_types ? (column.type->getSQLCompatibleName()) : (column.type->getName())); res_columns[res_index++]->insert(column.type->getName());
if (columns_mask[src_index++]) if (columns_mask[src_index++])
res_columns[res_index++]->insert(position); res_columns[res_index++]->insert(position);
@ -285,7 +284,6 @@ private:
ColumnPtr tables; ColumnPtr tables;
Storages storages; Storages storages;
ClientInfo::Interface client_info_interface; ClientInfo::Interface client_info_interface;
bool use_mysql_types;
size_t db_table_num = 0; size_t db_table_num = 0;
size_t total_tables; size_t total_tables;
std::shared_ptr<const ContextAccess> access; std::shared_ptr<const ContextAccess> access;