Addressed review feedback

This commit is contained in:
chen768959 2023-08-15 03:25:32 +08:00
parent 5719505df6
commit 74485f3e52
6 changed files with 19 additions and 75 deletions

View File

@ -1,7 +1,6 @@
#pragma once
#include <type_traits>
#include <utility>
#include <Core/Field.h>
#include <DataTypes/DataTypeNumberBase.h>
#include <DataTypes/Serializations/SerializationNumber.h>
@ -10,21 +9,10 @@
namespace DB
{
using DataTypes = std::vector<DataTypePtr>;
template <typename T>
class DataTypeNumber final : public DataTypeNumberBase<T>
class DataTypeNumber : public DataTypeNumberBase<T>
{
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 canBeUsedAsVersion() const override { return true; }
@ -40,23 +28,10 @@ public:
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
{
return std::make_shared<SerializationNumber<T>>();
}
private:
DataTypePtr opposite_sign_data_type;
bool has_opposite_sign_data_type = false;
};
using DataTypeUInt8 = DataTypeNumber<UInt8>;

View File

@ -3,6 +3,7 @@
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesNumberWithOpposite.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeString.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<UInt16>::max()) return std::make_shared<DataTypeUInt16>();
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>();
}
@ -140,7 +141,7 @@ DataTypePtr FieldToDataType<on_error>::operator() (const Array & x) const
for (const Field & elem : x)
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>

View File

@ -71,12 +71,6 @@ DataTypePtr IDataType::promoteNumericType() const
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
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of type {} in memory is not of fixed size.", getName());

View File

@ -159,15 +159,6 @@ public:
*/
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.
* This should be overridden if data type default value differs from column default value (example: Enum data types).
*/

View File

@ -16,6 +16,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesNumberWithOpposite.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
@ -201,7 +202,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types)
}
template <LeastSupertypeOnError on_error>
DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids)
DataTypePtr getLeastSupertype(const DataTypes & types)
{
/// Trivial cases
@ -592,8 +593,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids)
/// 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);
if (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);
}
// 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)
{
auto is_signed_int = [](const TypeIndex & type_id)
{
switch (type_id)
{
case TypeIndex::Int8:
case TypeIndex::Int16:
case TypeIndex::Int32:
case TypeIndex::Int64:
return true;
default:
return false;
}
};
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;
bool has_signed_int = false;
bool has_uint64_and_has_opposite = false;
bool has_opposite = false;
TypeIndexSet opposite_type_ids;
// Determine whether UInt64 in type_ids needs to change its sign.
for (const auto & type : types)
{
auto type_id = type->getTypeId();
if (!has_signed_int)
has_signed_int = is_signed_int(type_id);
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;
break ;
has_opposite = true;
opposite_type_ids.insert(uint64_with_opposite->getOppositeSignDataType()->getTypeId());
}else
{
has_uint64_and_has_opposite = true;
opposite_type_ids.insert(type->oppositeSignDataType()->getTypeId());
}
return;
}
}
if (has_uint64_and_has_opposite && has_signed_int)
if (has_opposite)
{
type_ids.erase(TypeIndex::UInt64);
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);
}
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);
}

View File

@ -18,7 +18,7 @@ enum class LeastSupertypeOnError
* Examples: there is no least common supertype for Array(UInt8), Int8.
*/
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.
/// All types can be casted to String, because they can be serialized to String.