mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 09:22:05 +00:00
Addressed review feedback
This commit is contained in:
parent
5719505df6
commit
74485f3e52
@ -1,7 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <type_traits>
|
#include <type_traits>
|
||||||
#include <utility>
|
|
||||||
#include <Core/Field.h>
|
#include <Core/Field.h>
|
||||||
#include <DataTypes/DataTypeNumberBase.h>
|
#include <DataTypes/DataTypeNumberBase.h>
|
||||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||||
@ -10,21 +9,10 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using DataTypes = std::vector<DataTypePtr>;
|
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
class DataTypeNumber final : public DataTypeNumberBase<T>
|
class DataTypeNumber : public DataTypeNumberBase<T>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
DataTypeNumber() = default;
|
|
||||||
|
|
||||||
explicit DataTypeNumber(DataTypePtr opposite_sign_data_type_)
|
|
||||||
: DataTypeNumberBase<T>()
|
|
||||||
, opposite_sign_data_type(std::move(opposite_sign_data_type_))
|
|
||||||
, has_opposite_sign_data_type(true)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }
|
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }
|
||||||
|
|
||||||
bool canBeUsedAsVersion() const override { return true; }
|
bool canBeUsedAsVersion() const override { return true; }
|
||||||
@ -40,23 +28,10 @@ public:
|
|||||||
return std::make_shared<PromotedType>();
|
return std::make_shared<PromotedType>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool hasOppositeSignDataType() const override { return has_opposite_sign_data_type; }
|
|
||||||
DataTypePtr oppositeSignDataType() const override
|
|
||||||
{
|
|
||||||
if (!has_opposite_sign_data_type)
|
|
||||||
IDataType::oppositeSignDataType();
|
|
||||||
|
|
||||||
return opposite_sign_data_type;
|
|
||||||
}
|
|
||||||
|
|
||||||
SerializationPtr doGetDefaultSerialization() const override
|
SerializationPtr doGetDefaultSerialization() const override
|
||||||
{
|
{
|
||||||
return std::make_shared<SerializationNumber<T>>();
|
return std::make_shared<SerializationNumber<T>>();
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
|
||||||
DataTypePtr opposite_sign_data_type;
|
|
||||||
bool has_opposite_sign_data_type = false;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
using DataTypeUInt8 = DataTypeNumber<UInt8>;
|
using DataTypeUInt8 = DataTypeNumber<UInt8>;
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <DataTypes/DataTypeMap.h>
|
#include <DataTypes/DataTypeMap.h>
|
||||||
#include <DataTypes/DataTypeObject.h>
|
#include <DataTypes/DataTypeObject.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <DataTypes/DataTypesNumberWithOpposite.h>
|
||||||
#include <DataTypes/DataTypesDecimal.h>
|
#include <DataTypes/DataTypesDecimal.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
@ -36,7 +37,7 @@ DataTypePtr FieldToDataType<on_error>::operator() (const UInt64 & x) const
|
|||||||
if (x <= std::numeric_limits<UInt8>::max()) return std::make_shared<DataTypeUInt8>();
|
if (x <= std::numeric_limits<UInt8>::max()) return std::make_shared<DataTypeUInt8>();
|
||||||
if (x <= std::numeric_limits<UInt16>::max()) return std::make_shared<DataTypeUInt16>();
|
if (x <= std::numeric_limits<UInt16>::max()) return std::make_shared<DataTypeUInt16>();
|
||||||
if (x <= std::numeric_limits<UInt32>::max()) return std::make_shared<DataTypeUInt32>();
|
if (x <= std::numeric_limits<UInt32>::max()) return std::make_shared<DataTypeUInt32>();
|
||||||
if (x <= std::numeric_limits<Int64>::max()) return std::make_shared<DataTypeUInt64>(std::make_shared<DataTypeInt64>());
|
if (x <= std::numeric_limits<Int64>::max()) return std::make_shared<DataTypeUInt64WithOpposite>(std::make_shared<DataTypeInt64>());
|
||||||
return std::make_shared<DataTypeUInt64>();
|
return std::make_shared<DataTypeUInt64>();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -140,7 +141,7 @@ DataTypePtr FieldToDataType<on_error>::operator() (const Array & x) const
|
|||||||
for (const Field & elem : x)
|
for (const Field & elem : x)
|
||||||
element_types.emplace_back(applyVisitor(*this, elem));
|
element_types.emplace_back(applyVisitor(*this, elem));
|
||||||
|
|
||||||
return std::make_shared<DataTypeArray>(getLeastSupertype<on_error>(element_types, true));
|
return std::make_shared<DataTypeArray>(getLeastSupertype<on_error>(element_types));
|
||||||
}
|
}
|
||||||
|
|
||||||
template <LeastSupertypeOnError on_error>
|
template <LeastSupertypeOnError on_error>
|
||||||
|
@ -71,12 +71,6 @@ DataTypePtr IDataType::promoteNumericType() const
|
|||||||
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED, "Data type {} can't be promoted.", getName());
|
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED, "Data type {} can't be promoted.", getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr IDataType::oppositeSignDataType() const
|
|
||||||
{
|
|
||||||
// TODO: Should use ErrorCodes::OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND.
|
|
||||||
throw Exception(702, "Opposite sign data type not found for {}.", getName());
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t IDataType::getSizeOfValueInMemory() const
|
size_t IDataType::getSizeOfValueInMemory() const
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of type {} in memory is not of fixed size.", getName());
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of type {} in memory is not of fixed size.", getName());
|
||||||
|
@ -159,15 +159,6 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual DataTypePtr promoteNumericType() const;
|
virtual DataTypePtr promoteNumericType() const;
|
||||||
|
|
||||||
/** The data type has an opposite sign DataTypePtr type.
|
|
||||||
* Data types that can have an opposite sign are typically signed or unsigned types.
|
|
||||||
*/
|
|
||||||
virtual bool hasOppositeSignDataType() const { return false; }
|
|
||||||
|
|
||||||
/** Return the opposite sign data type of the current data type. Throw an exception if `hasOppositeSignDataType() == false`.
|
|
||||||
*/
|
|
||||||
virtual DataTypePtr oppositeSignDataType() const;
|
|
||||||
|
|
||||||
/** Directly insert default value into a column. Default implementation use method IColumn::insertDefault.
|
/** Directly insert default value into a column. Default implementation use method IColumn::insertDefault.
|
||||||
* This should be overridden if data type default value differs from column default value (example: Enum data types).
|
* This should be overridden if data type default value differs from column default value (example: Enum data types).
|
||||||
*/
|
*/
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
#include <DataTypes/DataTypeDateTime.h>
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
#include <DataTypes/DataTypeDateTime64.h>
|
#include <DataTypes/DataTypeDateTime64.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <DataTypes/DataTypesNumberWithOpposite.h>
|
||||||
#include <DataTypes/DataTypesDecimal.h>
|
#include <DataTypes/DataTypesDecimal.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
|
||||||
@ -201,7 +202,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types)
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <LeastSupertypeOnError on_error>
|
template <LeastSupertypeOnError on_error>
|
||||||
DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids)
|
DataTypePtr getLeastSupertype(const DataTypes & types)
|
||||||
{
|
{
|
||||||
/// Trivial cases
|
/// Trivial cases
|
||||||
|
|
||||||
@ -592,8 +593,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids)
|
|||||||
|
|
||||||
/// For numeric types, the most complicated part.
|
/// For numeric types, the most complicated part.
|
||||||
{
|
{
|
||||||
if (optimize_type_ids)
|
optimizeTypeIds(types, type_ids);
|
||||||
optimizeTypeIds(types, type_ids);
|
|
||||||
auto numeric_type = getNumericType<on_error>(type_ids);
|
auto numeric_type = getNumericType<on_error>(type_ids);
|
||||||
if (numeric_type)
|
if (numeric_type)
|
||||||
return numeric_type;
|
return numeric_type;
|
||||||
@ -603,49 +603,32 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids)
|
|||||||
return throwOrReturn<on_error>(types, "", ErrorCodes::NO_COMMON_TYPE);
|
return throwOrReturn<on_error>(types, "", ErrorCodes::NO_COMMON_TYPE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Convert the UInt64 type to Int64 in order to cover other signed_integer types and obtain the least super type of all ints.
|
||||||
|
// Example, UInt64(both Int64), Int8 = Int64, Int8.
|
||||||
void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids)
|
void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids)
|
||||||
{
|
{
|
||||||
auto is_signed_int = [](const TypeIndex & type_id)
|
if ((!type_ids.contains(TypeIndex::Int8) && !type_ids.contains(TypeIndex::Int16) && !type_ids.contains(TypeIndex::Int32) && !type_ids.contains(TypeIndex::Int64)) || !type_ids.contains(TypeIndex::UInt64))
|
||||||
{
|
return;
|
||||||
switch (type_id)
|
|
||||||
{
|
|
||||||
case TypeIndex::Int8:
|
|
||||||
case TypeIndex::Int16:
|
|
||||||
case TypeIndex::Int32:
|
|
||||||
case TypeIndex::Int64:
|
|
||||||
return true;
|
|
||||||
default:
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
bool has_signed_int = false;
|
bool has_opposite = false;
|
||||||
bool has_uint64_and_has_opposite = false;
|
|
||||||
TypeIndexSet opposite_type_ids;
|
TypeIndexSet opposite_type_ids;
|
||||||
|
|
||||||
// Determine whether UInt64 in type_ids needs to change its sign.
|
|
||||||
for (const auto & type : types)
|
for (const auto & type : types)
|
||||||
{
|
{
|
||||||
auto type_id = type->getTypeId();
|
auto type_id = type->getTypeId();
|
||||||
|
|
||||||
if (!has_signed_int)
|
|
||||||
has_signed_int = is_signed_int(type_id);
|
|
||||||
|
|
||||||
if (type_id == TypeIndex::UInt64)
|
if (type_id == TypeIndex::UInt64)
|
||||||
{
|
{
|
||||||
if (!type->hasOppositeSignDataType())
|
if (const auto * uint64_with_opposite = typeid_cast<const DataTypeUInt64WithOpposite *>(type.get()))
|
||||||
{
|
{
|
||||||
has_uint64_and_has_opposite = false;
|
has_opposite = true;
|
||||||
break ;
|
opposite_type_ids.insert(uint64_with_opposite->getOppositeSignDataType()->getTypeId());
|
||||||
}else
|
}else
|
||||||
{
|
return;
|
||||||
has_uint64_and_has_opposite = true;
|
|
||||||
opposite_type_ids.insert(type->oppositeSignDataType()->getTypeId());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (has_uint64_and_has_opposite && has_signed_int)
|
if (has_opposite)
|
||||||
{
|
{
|
||||||
type_ids.erase(TypeIndex::UInt64);
|
type_ids.erase(TypeIndex::UInt64);
|
||||||
type_ids.insert(opposite_type_ids.begin(), opposite_type_ids.end());
|
type_ids.insert(opposite_type_ids.begin(), opposite_type_ids.end());
|
||||||
@ -713,7 +696,7 @@ DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types)
|
|||||||
return getLeastSupertype<LeastSupertypeOnError::Null>(types);
|
return getLeastSupertype<LeastSupertypeOnError::Null>(types);
|
||||||
}
|
}
|
||||||
|
|
||||||
template DataTypePtr getLeastSupertype<LeastSupertypeOnError::Throw>(const DataTypes & types, bool optimize_type_ids);
|
template DataTypePtr getLeastSupertype<LeastSupertypeOnError::Throw>(const DataTypes & types);
|
||||||
template DataTypePtr getLeastSupertype<LeastSupertypeOnError::Throw>(const TypeIndexSet & types);
|
template DataTypePtr getLeastSupertype<LeastSupertypeOnError::Throw>(const TypeIndexSet & types);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -18,7 +18,7 @@ enum class LeastSupertypeOnError
|
|||||||
* Examples: there is no least common supertype for Array(UInt8), Int8.
|
* Examples: there is no least common supertype for Array(UInt8), Int8.
|
||||||
*/
|
*/
|
||||||
template <LeastSupertypeOnError on_error = LeastSupertypeOnError::Throw>
|
template <LeastSupertypeOnError on_error = LeastSupertypeOnError::Throw>
|
||||||
DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids = false);
|
DataTypePtr getLeastSupertype(const DataTypes & types);
|
||||||
|
|
||||||
/// Same as above but return String type instead of throwing exception.
|
/// Same as above but return String type instead of throwing exception.
|
||||||
/// All types can be casted to String, because they can be serialized to String.
|
/// All types can be casted to String, because they can be serialized to String.
|
||||||
|
Loading…
Reference in New Issue
Block a user