mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Revert "Add option to getMostSubtype/getLeastSupertype to not throw exception"
This reverts commit c4654d6b45
.
This commit is contained in:
parent
6172cc1676
commit
bf9cb8482b
@ -47,47 +47,19 @@ namespace
|
||||
|
||||
return res.str();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
DataTypePtr wrapWithType(DataTypePtr type)
|
||||
{
|
||||
if (type == nullptr)
|
||||
return nullptr;
|
||||
return std::make_shared<T>(type);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
DataTypePtr wrapWithType(DataTypePtr type1, DataTypePtr type2)
|
||||
{
|
||||
if (type1 == nullptr || type2 == nullptr)
|
||||
return nullptr;
|
||||
return std::make_shared<T>(type1, type2);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
DataTypePtr wrapWithType(const DataTypes & types)
|
||||
{
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (type == nullptr)
|
||||
return nullptr;
|
||||
}
|
||||
return std::make_shared<T>(types);
|
||||
}
|
||||
}
|
||||
|
||||
DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_type)
|
||||
|
||||
DataTypePtr getLeastSupertype(const DataTypes & types)
|
||||
{
|
||||
auto on_error = [throw_on_no_common_type](const String & msg) -> DataTypePtr
|
||||
{
|
||||
if (throw_on_no_common_type)
|
||||
throw Exception(msg, ErrorCodes::NO_COMMON_TYPE);
|
||||
return nullptr;
|
||||
};
|
||||
/// Trivial cases
|
||||
|
||||
if (types.empty())
|
||||
return std::make_shared<DataTypeNothing>();
|
||||
|
||||
if (types.size() == 1)
|
||||
return types[0];
|
||||
|
||||
/// All types are equal
|
||||
{
|
||||
bool all_equal = true;
|
||||
@ -116,7 +88,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
non_nothing_types.emplace_back(type);
|
||||
|
||||
if (non_nothing_types.size() < types.size())
|
||||
return getLeastSupertype(non_nothing_types, throw_on_no_common_type);
|
||||
return getLeastSupertype(non_nothing_types);
|
||||
}
|
||||
|
||||
/// For Arrays
|
||||
@ -141,9 +113,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (have_array)
|
||||
{
|
||||
if (!all_arrays)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are Array and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Array and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
return wrapWithType<DataTypeArray>(getLeastSupertype(nested_types, throw_on_no_common_type));
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype(nested_types));
|
||||
}
|
||||
}
|
||||
|
||||
@ -167,7 +139,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
nested_types[elem_idx].reserve(types.size());
|
||||
}
|
||||
else if (tuple_size != type_tuple->getElements().size())
|
||||
return on_error(getExceptionMessagePrefix(types) + " because Tuples have different sizes");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because Tuples have different sizes", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
have_tuple = true;
|
||||
|
||||
@ -181,13 +153,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (have_tuple)
|
||||
{
|
||||
if (!all_tuples)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
DataTypes common_tuple_types(tuple_size);
|
||||
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
|
||||
common_tuple_types[elem_idx] = getLeastSupertype(nested_types[elem_idx], throw_on_no_common_type);
|
||||
common_tuple_types[elem_idx] = getLeastSupertype(nested_types[elem_idx]);
|
||||
|
||||
return wrapWithType<DataTypeTuple>(common_tuple_types);
|
||||
return std::make_shared<DataTypeTuple>(common_tuple_types);
|
||||
}
|
||||
}
|
||||
|
||||
@ -215,9 +187,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (have_maps)
|
||||
{
|
||||
if (!all_maps)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are Maps and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Maps and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
return wrapWithType<DataTypeMap>(getLeastSupertype(key_types, throw_on_no_common_type), getLeastSupertype(value_types, throw_on_no_common_type));
|
||||
return std::make_shared<DataTypeMap>(getLeastSupertype(key_types), getLeastSupertype(value_types));
|
||||
}
|
||||
}
|
||||
|
||||
@ -248,9 +220,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (have_low_cardinality)
|
||||
{
|
||||
if (have_not_low_cardinality)
|
||||
return getLeastSupertype(nested_types, throw_on_no_common_type);
|
||||
return getLeastSupertype(nested_types);
|
||||
else
|
||||
return wrapWithType<DataTypeLowCardinality>(getLeastSupertype(nested_types, throw_on_no_common_type));
|
||||
return std::make_shared<DataTypeLowCardinality>(getLeastSupertype(nested_types));
|
||||
}
|
||||
}
|
||||
|
||||
@ -276,7 +248,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
|
||||
if (have_nullable)
|
||||
{
|
||||
return wrapWithType<DataTypeNullable>(getLeastSupertype(nested_types, throw_on_no_common_type));
|
||||
return std::make_shared<DataTypeNullable>(getLeastSupertype(nested_types));
|
||||
}
|
||||
}
|
||||
|
||||
@ -296,7 +268,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
{
|
||||
bool all_strings = type_ids.size() == (have_string + have_fixed_string);
|
||||
if (!all_strings)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are String/FixedString and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are String/FixedString and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
@ -312,7 +284,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
{
|
||||
bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime + have_datetime64);
|
||||
if (!all_date_or_datetime)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are Date/DateTime/DateTime64 and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Date/DateTime/DateTime64 and some of them are not",
|
||||
ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
if (have_datetime64 == 0)
|
||||
{
|
||||
@ -372,7 +345,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
}
|
||||
|
||||
if (num_supported != type_ids.size())
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them have no lossless conversion to Decimal");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them have no lossless conversion to Decimal",
|
||||
ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
UInt32 max_scale = 0;
|
||||
for (const auto & type : types)
|
||||
@ -394,8 +368,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
}
|
||||
|
||||
if (min_precision > DataTypeDecimal<Decimal128>::maxPrecision())
|
||||
return on_error(getExceptionMessagePrefix(types) + " because the least supertype is Decimal("
|
||||
+ toString(min_precision) + ',' + toString(max_scale) + ')');
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because the least supertype is Decimal("
|
||||
+ toString(min_precision) + ',' + toString(max_scale) + ')',
|
||||
ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
if (have_decimal128 || min_precision > DataTypeDecimal<Decimal64>::maxPrecision())
|
||||
return std::make_shared<DataTypeDecimal<Decimal128>>(DataTypeDecimal<Decimal128>::maxPrecision(), max_scale);
|
||||
@ -456,7 +431,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || max_mantissa_bits_of_floating)
|
||||
{
|
||||
if (!all_numbers)
|
||||
return on_error(getExceptionMessagePrefix(types) + " because some of them are numbers and some of them are not");
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
/// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit.
|
||||
/// Example, common of Int32, UInt32 = Int64.
|
||||
@ -471,10 +446,11 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
if (min_bit_width_of_integer != 64)
|
||||
++min_bit_width_of_integer;
|
||||
else
|
||||
return on_error(
|
||||
throw Exception(
|
||||
getExceptionMessagePrefix(types)
|
||||
+ " because some of them are signed integers and some are unsigned integers,"
|
||||
" but there is no signed integer type, that can exactly represent all required unsigned integer values");
|
||||
" but there is no signed integer type, that can exactly represent all required unsigned integer values",
|
||||
ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
/// If the result must be floating.
|
||||
@ -486,9 +462,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
else if (min_mantissa_bits <= 53)
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
else
|
||||
return on_error(getExceptionMessagePrefix(types)
|
||||
throw Exception(getExceptionMessagePrefix(types)
|
||||
+ " because some of them are integers and some are floating point,"
|
||||
" but there is no floating point type, that can exactly represent all required integers");
|
||||
" but there is no floating point type, that can exactly represent all required integers", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
/// If the result must be signed integer.
|
||||
@ -507,9 +483,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
else if (min_bit_width_of_integer <= 256)
|
||||
return std::make_shared<DataTypeInt256>();
|
||||
else
|
||||
return on_error(getExceptionMessagePrefix(types)
|
||||
throw Exception(getExceptionMessagePrefix(types)
|
||||
+ " because some of them are signed integers and some are unsigned integers,"
|
||||
" but there is no signed integer type, that can exactly represent all required unsigned integer values");
|
||||
" but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
/// All unsigned.
|
||||
@ -527,14 +503,14 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_t
|
||||
else if (min_bit_width_of_integer <= 256)
|
||||
return std::make_shared<DataTypeUInt256>();
|
||||
else
|
||||
return on_error("Logical error: " + getExceptionMessagePrefix(types)
|
||||
+ " but as all data types are unsigned integers, we must have found maximum unsigned integer type");
|
||||
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
||||
+ " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
|
||||
return on_error(getExceptionMessagePrefix(types));
|
||||
throw Exception(getExceptionMessagePrefix(types), ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,15 +2,16 @@
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Get data type that covers all possible values of passed data types.
|
||||
* If there is no such data type, throws an exception or return nullptr.
|
||||
* If there is no such data type, throws an exception.
|
||||
*
|
||||
* Examples: least common supertype for UInt8, Int8 - Int16.
|
||||
* Examples: there is no least common supertype for Array(UInt8), Int8.
|
||||
*/
|
||||
DataTypePtr getLeastSupertype(const DataTypes & types, bool throw_on_no_common_type = true);
|
||||
DataTypePtr getLeastSupertype(const DataTypes & types);
|
||||
|
||||
}
|
||||
|
@ -23,8 +23,7 @@ extern const int NO_COMMON_TYPE;
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
String getExceptionMessage(const DataTypes & types, const std::string & reason)
|
||||
String getExceptionMessagePrefix(const DataTypes & types)
|
||||
{
|
||||
WriteBufferFromOwnString res;
|
||||
res << "There is no subtype for types ";
|
||||
@ -38,18 +37,19 @@ String getExceptionMessage(const DataTypes & types, const std::string & reason)
|
||||
|
||||
res << type->getName();
|
||||
}
|
||||
res << reason;
|
||||
|
||||
return res.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type, bool force_support_conversion)
|
||||
|
||||
DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_nothing, bool force_support_conversion)
|
||||
{
|
||||
auto get_nothing_or_throw = [throw_on_no_common_type, & types](const std::string & reason) -> DataTypePtr
|
||||
auto get_nothing_or_throw = [throw_if_result_is_nothing, & types](const std::string & reason)
|
||||
{
|
||||
if (throw_on_no_common_type)
|
||||
throw Exception(getExceptionMessage(types, reason), ErrorCodes::NO_COMMON_TYPE);
|
||||
if (throw_if_result_is_nothing)
|
||||
throw Exception(getExceptionMessagePrefix(types) + reason, ErrorCodes::NO_COMMON_TYPE);
|
||||
return std::make_shared<DataTypeNothing>();
|
||||
};
|
||||
|
||||
@ -57,14 +57,14 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
|
||||
if (types.empty())
|
||||
{
|
||||
if (throw_on_no_common_type)
|
||||
if (throw_if_result_is_nothing)
|
||||
throw Exception("There is no common type for empty type list", ErrorCodes::NO_COMMON_TYPE);
|
||||
return std::make_shared<DataTypeNothing>();
|
||||
}
|
||||
|
||||
if (types.size() == 1)
|
||||
{
|
||||
if (throw_on_no_common_type && typeid_cast<const DataTypeNothing *>(types[0].get()))
|
||||
if (throw_if_result_is_nothing && typeid_cast<const DataTypeNothing *>(types[0].get()))
|
||||
throw Exception("There is no common type for type Nothing", ErrorCodes::NO_COMMON_TYPE);
|
||||
return types[0];
|
||||
}
|
||||
@ -118,7 +118,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
if (!all_arrays)
|
||||
return get_nothing_or_throw(" because some of them are Array and some of them are not");
|
||||
|
||||
return std::make_shared<DataTypeArray>(getMostSubtype(nested_types, throw_on_no_common_type, force_support_conversion));
|
||||
return std::make_shared<DataTypeArray>(getMostSubtype(nested_types, false, force_support_conversion));
|
||||
}
|
||||
}
|
||||
|
||||
@ -161,7 +161,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
DataTypes common_tuple_types(tuple_size);
|
||||
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
|
||||
common_tuple_types[elem_idx] =
|
||||
getMostSubtype(nested_types[elem_idx], throw_on_no_common_type, force_support_conversion);
|
||||
getMostSubtype(nested_types[elem_idx], throw_if_result_is_nothing, force_support_conversion);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(common_tuple_types);
|
||||
}
|
||||
@ -192,9 +192,9 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
if (have_nullable)
|
||||
{
|
||||
if (all_nullable || force_support_conversion)
|
||||
return std::make_shared<DataTypeNullable>(getMostSubtype(nested_types, throw_on_no_common_type, force_support_conversion));
|
||||
return std::make_shared<DataTypeNullable>(getMostSubtype(nested_types, false, force_support_conversion));
|
||||
|
||||
return getMostSubtype(nested_types, throw_on_no_common_type, force_support_conversion);
|
||||
return getMostSubtype(nested_types, throw_if_result_is_nothing, force_support_conversion);
|
||||
}
|
||||
}
|
||||
|
||||
@ -309,10 +309,8 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
else if (min_mantissa_bits_of_floating <= 53)
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::NO_COMMON_TYPE,
|
||||
"Logical error: " +
|
||||
getExceptionMessage(types, " but as all data types are floats, we must have found maximum float type"));
|
||||
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
||||
+ " but as all data types are floats, we must have found maximum float type", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
/// If there are signed and unsigned types of same bit-width, the result must be unsigned number.
|
||||
@ -328,10 +326,8 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
else if (min_bits_of_unsigned_integer <= 64)
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::NO_COMMON_TYPE,
|
||||
"Logical error: " +
|
||||
getExceptionMessage(types, " but as all data types are integers, we must have found maximum unsigned integer type"));
|
||||
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
||||
+ " but as all data types are integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
|
||||
/// All signed.
|
||||
@ -345,10 +341,8 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_on_no_common_type
|
||||
else if (min_bits_of_signed_integer <= 64)
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::NO_COMMON_TYPE,
|
||||
"Logical error: " +
|
||||
getExceptionMessage(types, " but as all data types are integers, we must have found maximum signed integer type"));
|
||||
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
||||
+ " but as all data types are integers, we must have found maximum signed integer type", ErrorCodes::NO_COMMON_TYPE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -12,9 +12,8 @@ namespace DB
|
||||
* Examples: most common subtype for Array(UInt8), Int8 is Nothing
|
||||
*
|
||||
* If force_support_conversion is true, returns type which may be used to convert each argument to.
|
||||
* Example: most common subtype for Array(UInt8) and Array(Nullable(Int32)) is Array(Nullable(UInt8)) if force_support_conversion is true.
|
||||
* Example: most common subtype for Array(UInt8) and Array(Nullable(Int32)) is Array(Nullable(UInt8) if force_support_conversion is true.
|
||||
*/
|
||||
DataTypePtr
|
||||
getMostSubtype(const DataTypes & types, bool throw_on_no_common_type = true, bool force_support_conversion = false);
|
||||
DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_nothing = false, bool force_support_conversion = false);
|
||||
|
||||
}
|
||||
|
@ -13,9 +13,7 @@
|
||||
|
||||
#include <Interpreters/DictionaryReader.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
@ -423,14 +421,19 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
|
||||
if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second))
|
||||
continue;
|
||||
|
||||
/// TODO(vdimir): use getMostSubtype if possible
|
||||
auto common_type = DB::getLeastSupertype({ltype->second, rtype->second}, false);
|
||||
if (common_type == nullptr || isNothing(common_type))
|
||||
DataTypePtr common_type;
|
||||
try
|
||||
{
|
||||
/// TODO(vdimir): use getMostSubtype if possible
|
||||
common_type = DB::getLeastSupertype({ltype->second, rtype->second});
|
||||
}
|
||||
catch (DB::Exception & ex)
|
||||
{
|
||||
throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Can't infer supertype for joined columns: {}: {} at left, {}: {} at right.",
|
||||
"Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}",
|
||||
key_names_left[i], ltype->second->getName(),
|
||||
key_names_right[i], rtype->second->getName());
|
||||
key_names_right[i], rtype->second->getName(),
|
||||
ex.message());
|
||||
}
|
||||
|
||||
if (!allow_right && !common_type->equals(*rtype->second))
|
||||
|
Loading…
Reference in New Issue
Block a user