bug fix, refactoring IPv6 to struct with overridden comparison

This commit is contained in:
Yakov Olkhovskiy 2022-11-22 12:57:19 +00:00
parent ee3ce9de70
commit 988b030bfd
7 changed files with 85 additions and 91 deletions

View File

@ -2,9 +2,54 @@
#include <base/strong_typedef.h>
#include <base/extended_types.h>
#include <Common/memcmpSmall.h>
#include "base/Decimal.h"
namespace DB
{
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
using IPv6 = StrongTypedef<UInt128, struct IPv6Tag>;
struct IPv6 : StrongTypedef<UInt128, struct IPv6Tag>
{
constexpr IPv6() = default;
constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {}
constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {}
IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; }
IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; }
bool operator<(const IPv6 & rhs) const
{
return
memcmp16(
reinterpret_cast<const unsigned char *>(toUnderType().items),
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
) < 0;
}
bool operator>(const IPv6 & rhs) const
{
return
memcmp16(
reinterpret_cast<const unsigned char *>(toUnderType().items),
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
) > 0;
}
bool operator==(const IPv6 & rhs) const
{
return
memcmp16(
reinterpret_cast<const unsigned char *>(toUnderType().items),
reinterpret_cast<const unsigned char *>(rhs.toUnderType().items)
) == 0;
}
bool operator<=(const IPv6 & rhs) const { return !operator>(rhs); }
bool operator>=(const IPv6 & rhs) const { return !operator<(rhs); }
bool operator!=(const IPv6 & rhs) const { return !operator==(rhs); }
};
}

View File

@ -60,7 +60,7 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf)
case Field::Types::IPv6:
{
IPv6 value;
readBinary(value, buf);
readBinary(value.toUnderType(), buf);
return value;
}
case Field::Types::Int64:

View File

@ -514,8 +514,8 @@ public:
case Types::Int128: return get<Int128>() <= rhs.get<Int128>();
case Types::Int256: return get<Int256>() <= rhs.get<Int256>();
case Types::UUID: return get<UUID>().toUnderType() <= rhs.get<UUID>().toUnderType();
case Types::IPv4: return get<IPv4>().toUnderType() <= rhs.get<IPv4>().toUnderType();
case Types::IPv6: return get<IPv6>().toUnderType() <= rhs.get<IPv6>().toUnderType();
case Types::IPv4: return get<IPv4>() <= rhs.get<IPv4>();
case Types::IPv6: return get<IPv6>() <= rhs.get<IPv6>();
case Types::Float64: return get<Float64>() <= rhs.get<Float64>();
case Types::String: return get<String>() <= rhs.get<String>();
case Types::Array: return get<Array>() <= rhs.get<Array>();

View File

@ -83,7 +83,7 @@ public:
void deserializeBinary(Field & field, ReadBuffer & istr) const override
{
IPv x;
readBinary(x, istr);
readBinary(x.toUnderType(), istr);
field = NearestFieldType<IPv>(x);
}
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
@ -92,8 +92,8 @@ public:
}
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override
{
IPv4 x;
readBinary(x, istr);
IPv x;
readBinary(x.toUnderType(), istr);
assert_cast<ColumnVector<IPv> &>(column).getData().push_back(x);
}
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override

View File

@ -511,11 +511,17 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!checkAndGetDataType<DataTypeUInt32>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() +
" of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto * dt_uint32 = checkAndGetDataType<DataTypeUInt32>(arguments[0].get());
const auto * dt_ipv4 = checkAndGetDataType<DataTypeIPv4>(arguments[0].get());
if (!dt_uint32 && !dt_ipv4)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}", arguments[0]->getName(), getName()
);
return std::make_shared<DataTypeFixedString>(16);
if (dt_uint32)
return std::make_shared<DataTypeFixedString>(16);
return std::make_shared<DataTypeIPv6>();
}
bool useDefaultImplementationForConstants() const override { return true; }
@ -525,7 +531,22 @@ public:
const auto & col_type_name = arguments[0];
const ColumnPtr & column = col_type_name.column;
if (const auto * col_in = typeid_cast<const ColumnUInt32 *>(column.get()))
if (const auto * col_in = checkAndGetColumn<ColumnIPv4>(*column))
{
auto col_res = ColumnIPv6::create();
auto & vec_res = col_res->getData();
vec_res.resize(col_in->size());
const auto & vec_in = col_in->getData();
for (size_t i = 0; i < vec_res.size(); ++i)
mapIPv4ToIPv6(vec_in[i], reinterpret_cast<UInt8 *>(&vec_res[i].toUnderType()));
return col_res;
}
if (const auto * col_in = checkAndGetColumn<ColumnUInt32>(*column))
{
auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
@ -539,10 +560,11 @@ public:
return col_res;
}
else
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()
);
}
private:

View File

@ -842,72 +842,6 @@ private:
return res;
}
ColumnPtr executeIPv6(const IColumn * c0, const IColumn * c1) const
{
const ColumnIPv6 * c0_string = checkAndGetColumn<ColumnIPv6>(c0);
const ColumnIPv6 * c1_string = checkAndGetColumn<ColumnIPv6>(c1);
const ColumnConst * c0_const = checkAndGetColumnConst<ColumnIPv6>(c0);
const ColumnConst * c1_const = checkAndGetColumnConst<ColumnIPv6>(c1);
if (!((c0_string || c0_const) && (c1_string || c1_const)))
return nullptr;
const ColumnIPv6::Container * c0_const_chars = nullptr;
const ColumnIPv6::Container * c1_const_chars = nullptr;
if (c0_const)
{
const ColumnIPv6 * c0_const_string = checkAndGetColumn<ColumnIPv6>(&c0_const->getDataColumn());
if (c0_const_string)
c0_const_chars = &c0_const_string->getData();
else
throw Exception("Logical error: ColumnConst contains not String nor FixedString column", ErrorCodes::ILLEGAL_COLUMN);
}
if (c1_const)
{
const ColumnIPv6 * c1_const_string = checkAndGetColumn<ColumnIPv6>(&c1_const->getDataColumn());
if (c1_const_string)
c1_const_chars = &c1_const_string->getData();
else
throw Exception("Logical error: ColumnConst contains not String nor FixedString column", ErrorCodes::ILLEGAL_COLUMN);
}
using IPv6Impl = IPv6ComparisonImpl<Op<int, int>>;
if (c0_const && c1_const)
{
auto res = executeIPv6(&c0_const->getDataColumn(), &c1_const->getDataColumn());
if (!res)
return nullptr;
return ColumnConst::create(res, c0_const->size());
}
else
{
auto c_res = ColumnUInt8::create();
ColumnUInt8::Container & vec_res = c_res->getData();
vec_res.resize(c0->size());
if (c0_string && c1_string)
IPv6Impl::ipv6_vector_ipv6_vector(c0_string->getData(), c1_string->getData(), c_res->getData());
else if (c0_string && c1_const)
IPv6Impl::ipv6_vector_constant(c0_string->getData(), *c1_const_chars, c_res->getData());
else if (c0_const && c1_string)
IPv6Impl::constant_ipv6_vector(*c0_const_chars, c1_string->getData(), c_res->getData());
else
throw Exception("Illegal columns "
+ c0->getName() + " and " + c1->getName()
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return c_res;
}
}
ColumnPtr executeString(const IColumn * c0, const IColumn * c1) const
{
const ColumnString * c0_string = checkAndGetColumn<ColumnString>(c0);
@ -1397,9 +1331,6 @@ public:
const bool left_is_float = which_left.isFloat();
const bool right_is_float = which_right.isFloat();
const bool left_is_ipv6 = which_left.isIPv6();
const bool right_is_ipv6 = which_right.isIPv6();
bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDate32() || which_left.isDateTime() || which_left.isDateTime64())
&& (which_right.isDate() || which_right.isDate32() || which_right.isDateTime() || which_right.isDateTime64());
@ -1431,10 +1362,6 @@ public:
{
return executeTuple(result_type, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
}
else if ((left_is_ipv6 && right_is_ipv6) && (res = executeIPv6(col_left_untyped, col_right_untyped)))
{
return res;
}
else if (left_is_string && right_is_string && (res = executeString(col_left_untyped, col_right_untyped)))
{
return res;

View File

@ -1021,7 +1021,7 @@ inline void parseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer & rb
{
IPv6 tmp;
readIPv6Text(tmp, rb);
x = tmp.toUnderType();
x = tmp;
}
template <typename DataType>
@ -1092,7 +1092,7 @@ inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer &
if (!tryReadIPv6Text(tmp, rb))
return false;
x = tmp.toUnderType();
x = tmp;
return true;
}