#include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; extern const int LOGICAL_ERROR; extern const int TYPE_MISMATCH; extern const int UNEXPECTED_DATA_AFTER_PARSED_VALUE; } /** Checking for a `Field from` of `From` type falls to a range of values of type `To`. * `From` and `To` - numeric types. They can be floating-point types. * `From` is one of UInt64, Int64, Float64, * whereas `To` can also be 8, 16, 32 bit. * * If falls into a range, then `from` is converted to the `Field` closest to the `To` type. * If not, return Field(Null). */ namespace { template Field convertNumericTypeImpl(const Field & from) { To result; if (!accurate::convertNumeric(from.get(), result)) return {}; return result; } template Field convertNumericType(const Field & from, const IDataType & type) { if (from.getType() == Field::Types::UInt64 || from.getType() == Field::Types::Bool) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::Int64) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::Float64) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::UInt128) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::Int128) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::UInt256) return convertNumericTypeImpl(from); if (from.getType() == Field::Types::Int256) return convertNumericTypeImpl(from); throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}", type.getName(), from.getType()); } template Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & type) { From value = from.get(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); T scaled_value = type.getScaleMultiplier() * T(static_cast(value)); return DecimalField(scaled_value, type.getScale()); } template Field convertStringToDecimalType(const Field & from, const DataTypeDecimal & type) { const String & str_value = from.get(); T value = type.parseFromString(str_value); return DecimalField(value, type.getScale()); } template Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal & type) { auto field = from.get>(); T value = convertDecimals, DataTypeDecimal>(field.getValue(), field.getScale(), type.getScale()); return DecimalField(value, type.getScale()); } template Field convertFloatToDecimalType(const Field & from, const DataTypeDecimal & type) { From value = from.get(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); //String sValue = convertFieldToString(from); //int fromScale = sValue.length()- sValue.find('.') - 1; UInt32 scale = type.getScale(); auto scaled_value = convertToDecimal, DataTypeDecimal>(value, scale); return DecimalField(scaled_value, scale); } template Field convertDecimalType(const Field & from, const To & type) { if (from.getType() == Field::Types::UInt64) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::Int64) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::UInt128) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::Int128) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::UInt256) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::Int256) return convertIntToDecimalType(from, type); if (from.getType() == Field::Types::String) return convertStringToDecimalType(from, type); if (from.getType() == Field::Types::Decimal32) return convertDecimalToDecimalType(from, type); if (from.getType() == Field::Types::Decimal64) return convertDecimalToDecimalType(from, type); if (from.getType() == Field::Types::Decimal128) return convertDecimalToDecimalType(from, type); if (from.getType() == Field::Types::Decimal256) return convertDecimalToDecimalType(from, type); if (from.getType() == Field::Types::Float64) return convertFloatToDecimalType(from, type); throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}", type.getName(), from.getType()); } Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint) { if (from_type_hint && from_type_hint->equals(type)) { return src; } WhichDataType which_type(type); WhichDataType which_from_type; if (from_type_hint) { which_from_type = WhichDataType(*from_type_hint); } /// Conversion between Date and DateTime and vice versa. if (which_type.isDate() && which_from_type.isDateTime()) { return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); } else if (which_type.isDate32() && which_from_type.isDateTime()) { return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); } else if (which_type.isDateTime() && which_from_type.isDate()) { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } else if (which_type.isDateTime() && which_from_type.isDate32()) { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } else if (which_type.isDateTime64() && which_from_type.isDate()) { const auto & date_time64_type = static_cast(type); const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.get())); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); } else if (which_type.isDateTime64() && which_from_type.isDate32()) { const auto & date_time64_type = static_cast(type); const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.get()))); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { if (which_type.isUInt8()) return convertNumericType(src, type); if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); if (which_type.isUInt64()) return convertNumericType(src, type); if (which_type.isUInt128()) return convertNumericType(src, type); if (which_type.isUInt256()) return convertNumericType(src, type); if (which_type.isInt8()) return convertNumericType(src, type); if (which_type.isInt16()) return convertNumericType(src, type); if (which_type.isInt32()) return convertNumericType(src, type); if (which_type.isInt64()) return convertNumericType(src, type); if (which_type.isInt128()) return convertNumericType(src, type); if (which_type.isInt256()) return convertNumericType(src, type); if (which_type.isFloat32()) return convertNumericType(src, type); if (which_type.isFloat64()) return convertNumericType(src, type); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (which_type.isEnum() && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64)) { /// Convert UInt64 or Int64 to Enum's value return dynamic_cast(type).castToValue(src); } if ((which_type.isDate() || which_type.isDateTime()) && src.getType() == Field::Types::UInt64) { /// We don't need any conversion UInt64 is under type of Date and DateTime return src; } if (which_type.isDate32() && src.getType() == Field::Types::Int64) { /// We don't need any conversion Int64 is under type of Date32 return src; } if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) { const auto & from_type = src.get(); const auto & to_type = static_cast(type); const auto scale_from = from_type.getScale(); const auto scale_to = to_type.getScale(); const auto scale_multiplier_diff = scale_from > scale_to ? from_type.getScaleMultiplier() / to_type.getScaleMultiplier() : to_type.getScaleMultiplier() / from_type.getScaleMultiplier(); if (scale_multiplier_diff == 1) /// Already in needed type. return src; /// in case if we need to make DateTime64(a) from DateTime64(b), a != b, we need to convert datetime value to the right scale const UInt64 value = scale_from > scale_to ? from_type.getValue().value / scale_multiplier_diff : from_type.getValue().value * scale_multiplier_diff; return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1), scale_to); } /// For toDate('xxx') in 1::Int64, we CAST `src` to UInt64, which may /// produce wrong result in some special cases. if (which_type.isDate() && src.getType() == Field::Types::Int64) { return convertNumericType(src, type); } /// For toDate32('xxx') in 1, we CAST `src` to Int64. Also, it may /// produce wrong result in some special cases. if (which_type.isDate32() && src.getType() == Field::Types::UInt64) { return convertNumericType(src, type); } if (which_type.isDateTime64() && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64 || src.getType() == Field::Types::Decimal64)) { const auto scale = static_cast(type).getScale(); const auto decimal_value = DecimalUtils::decimalFromComponents(applyVisitor(FieldVisitorConvertToNumber(), src), 0, scale); return Field(DecimalField(decimal_value, scale)); } if (which_type.isIPv4() && src.getType() == Field::Types::IPv4) { /// Already in needed type. return src; } if (which_type.isIPv4() && src.getType() == Field::Types::UInt64) { /// convert to UInt32 which is the underlying type for native IPv4 return convertNumericType(src, type); } } else if (which_type.isUUID() && src.getType() == Field::Types::UUID) { /// Already in needed type. return src; } else if (which_type.isIPv6()) { /// Already in needed type. if (src.getType() == Field::Types::IPv6) return src; /// Treat FixedString(16) as a binary representation of IPv6 if (which_from_type.isFixedString() && assert_cast(from_type_hint)->getN() == IPV6_BINARY_LENGTH) { const auto col = type.createColumn(); ReadBufferFromString in_buffer(src.get()); type.getDefaultSerialization()->deserializeBinary(*col, in_buffer, {}); return (*col)[0]; } } else if (which_type.isStringOrFixedString()) { if (src.getType() == Field::Types::String) { if (which_type.isFixedString()) { size_t n = assert_cast(type).getN(); const auto & src_str = src.get(); if (src_str.size() < n) { String src_str_extended = src_str; src_str_extended.resize(n); return src_str_extended; } } return src; } return applyVisitor(FieldVisitorToString(), src); } else if (const DataTypeArray * type_array = typeid_cast(&type)) { if (src.getType() == Field::Types::Array) { const Array & src_arr = src.get(); size_t src_arr_size = src_arr.size(); const auto & element_type = *(type_array->getNestedType()); bool have_unconvertible_element = false; Array res(src_arr_size); for (size_t i = 0; i < src_arr_size; ++i) { res[i] = convertFieldToType(src_arr[i], element_type); if (res[i].isNull() && !element_type.isNullable()) { // See the comment for Tuples below. have_unconvertible_element = true; } } return have_unconvertible_element ? Field(Null()) : Field(res); } } else if (const DataTypeTuple * type_tuple = typeid_cast(&type)) { if (src.getType() == Field::Types::Tuple) { const auto & src_tuple = src.get(); size_t src_tuple_size = src_tuple.size(); size_t dst_tuple_size = type_tuple->getElements().size(); if (dst_tuple_size != src_tuple_size) throw Exception(ErrorCodes::TYPE_MISMATCH, "Bad size of tuple in IN or VALUES section. " "Expected size: {}, actual size: {}", dst_tuple_size, src_tuple_size); Tuple res(dst_tuple_size); bool have_unconvertible_element = false; for (size_t i = 0; i < dst_tuple_size; ++i) { const auto & element_type = *(type_tuple->getElements()[i]); res[i] = convertFieldToType(src_tuple[i], element_type); if (!res[i].isNull() || element_type.isNullable()) continue; /* * Either the source element was Null, or the conversion did not * succeed, because the source and the requested types of the * element are compatible, but the value is not convertible * (e.g. trying to convert -1 from Int8 to UInt8). In these * cases, consider the whole tuple also compatible but not * convertible. According to the specification of this function, * we must return Null in this case. * * The following elements might be not even compatible, so it * makes sense to check them to detect user errors. Remember * that there is an unconvertible element, and try to process * the remaining ones. The convertFieldToType for each element * will throw if it detects incompatibility. */ have_unconvertible_element = true; } return have_unconvertible_element ? Field(Null()) : Field(res); } } else if (const DataTypeMap * type_map = typeid_cast(&type)) { if (src.getType() == Field::Types::Map) { const auto & key_type = *type_map->getKeyType(); const auto & value_type = *type_map->getValueType(); const auto & map = src.get(); size_t map_size = map.size(); Map res(map_size); bool have_unconvertible_element = false; for (size_t i = 0; i < map_size; ++i) { const auto & map_entry = map[i].get(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; Tuple updated_entry(2); updated_entry[0] = convertFieldToType(key, key_type); if (updated_entry[0].isNull() && !key_type.isNullable()) have_unconvertible_element = true; updated_entry[1] = convertFieldToType(value, value_type); if (updated_entry[1].isNull() && !value_type.isNullable()) have_unconvertible_element = true; res[i] = updated_entry; } return have_unconvertible_element ? Field(Null()) : Field(res); } } else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast(&type)) { if (src.getType() != Field::Types::AggregateFunctionState) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert {} to {}", src.getTypeName(), agg_func_type->getName()); const auto & name = src.get().name; if (agg_func_type->getName() != name) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert {} to {}", name, agg_func_type->getName()); return src; } else if (isObject(type)) { if (src.getType() == Field::Types::Object) return src; /// Already in needed type. const auto * from_type_tuple = typeid_cast(from_type_hint); if (src.getType() == Field::Types::Tuple && from_type_tuple && from_type_tuple->haveExplicitNames()) { const auto & names = from_type_tuple->getElementNames(); const auto & tuple = src.get(); if (names.size() != tuple.size()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Bad size of tuple in IN or VALUES section (while converting to Object). Expected size: {}, actual size: {}", names.size(), tuple.size()); Object object; for (size_t i = 0; i < names.size(); ++i) object[names[i]] = tuple[i]; return object; } if (src.getType() == Field::Types::Map) { Object object; const auto & map = src.get(); for (const auto & element : map) { const auto & map_entry = element.get(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; if (key.getType() != Field::Types::String) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert from Map with key of type {} to Object", key.getTypeName()); object[key.get()] = value; } return object; } } else if (const DataTypeVariant * type_variant = typeid_cast(&type)) { /// If we have type hint and Variant contains such type, no need to convert field. if (from_type_hint && type_variant->tryGetVariantDiscriminator(*from_type_hint)) return src; /// Create temporary column and check if we can insert this field to the variant. /// If we can insert, no need to convert anything. auto col = type_variant->createColumn(); if (col->tryInsert(src)) return src; } /// Conversion from string by parsing. if (src.getType() == Field::Types::String) { /// Promote data type to avoid overflows. Note that overflows in the largest data type are still possible. /// But don't promote Float32, since we want to keep the exact same value /// Also don't promote domain types (like bool) because we would otherwise use the serializer of the promoted type (e.g. UInt64 for /// bool, which does not allow 'true' and 'false' as input values) const IDataType * type_to_parse = &type; DataTypePtr holder; if (type.canBePromoted() && !which_type.isFloat32() && !type.getCustomSerialization()) { holder = type.promoteNumericType(); type_to_parse = holder.get(); } const auto col = type_to_parse->createColumn(); ReadBufferFromString in_buffer(src.get()); try { type_to_parse->getDefaultSerialization()->deserializeWholeText(*col, in_buffer, FormatSettings{}); } catch (Exception & e) { if (e.code() == ErrorCodes::UNEXPECTED_DATA_AFTER_PARSED_VALUE) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get(), type.getName()); e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); throw; } Field parsed = (*col)[0]; return convertFieldToType(parsed, type, from_type_hint); } throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}", type.getName(), src.getType()); } } Field convertFieldToType(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) { if (from_value.isNull()) return from_value; if (from_type_hint && from_type_hint->equals(to_type)) return from_value; if (const auto * low_cardinality_type = typeid_cast(&to_type)) return convertFieldToType(from_value, *low_cardinality_type->getDictionaryType(), from_type_hint); else if (const auto * nullable_type = typeid_cast(&to_type)) { const IDataType & nested_type = *nullable_type->getNestedType(); /// NULL remains NULL after any conversion. if (WhichDataType(nested_type).isNothing()) return {}; if (from_type_hint && from_type_hint->equals(nested_type)) return from_value; return convertFieldToTypeImpl(from_value, nested_type, from_type_hint); } else return convertFieldToTypeImpl(from_value, to_type, from_type_hint); } Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) { bool is_null = from_value.isNull(); if (is_null && !to_type.isNullable() && !to_type.isLowCardinalityNullable()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); Field converted = convertFieldToType(from_value, to_type, from_type_hint); if (!is_null && converted.isNull()) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Cannot convert value '{}'{}: it cannot be represented as {}", toString(from_value), from_type_hint ? " from " + from_type_hint->getName() : "", to_type.getName()); return converted; } template static bool decimalEqualsFloat(Field field, Float64 float_value) { auto decimal_field = field.get>(); auto decimal_to_float = DecimalUtils::convertTo(decimal_field.getValue(), decimal_field.getScale()); return decimal_to_float == float_value; } std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type) { Field result_value = convertFieldToType(from_value, to_type); if (Field::isDecimal(from_value.getType()) && Field::isDecimal(result_value.getType())) { bool is_equal = applyVisitor(FieldVisitorAccurateEquals{}, from_value, result_value); return is_equal ? result_value : std::optional{}; } if (from_value.getType() == Field::Types::Float64 && Field::isDecimal(result_value.getType())) { /// Convert back to Float64 and compare if (result_value.getType() == Field::Types::Decimal32) return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal64) return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal128) return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal256) return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); } return result_value; } }