Skip to content

Commit

Permalink
Merge pull request #12455 from amosbird/npc
Browse files Browse the repository at this point in the history
Nullable primary key with correct KeyCondition
  • Loading branch information
alexey-milovidov authored Jul 18, 2021
2 parents 9f529b0 + c56df3e commit b52411a
Show file tree
Hide file tree
Showing 37 changed files with 657 additions and 446 deletions.
4 changes: 4 additions & 0 deletions src/AggregateFunctions/AggregateFunctionSumMap.h
Original file line number Diff line number Diff line change
Expand Up @@ -459,6 +459,8 @@ class FieldVisitorMax : public StaticVisitor<bool>
explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {}

bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (NegativeInfinity &) const { throw Exception("Cannot compare -Inf", ErrorCodes::LOGICAL_ERROR); }
bool operator() (PositiveInfinity &) const { throw Exception("Cannot compare +Inf", ErrorCodes::LOGICAL_ERROR); }
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot compare AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }

bool operator() (Array & x) const { return compareImpl<Array>(x); }
Expand Down Expand Up @@ -494,6 +496,8 @@ class FieldVisitorMin : public StaticVisitor<bool>
explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {}

bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (NegativeInfinity &) const { throw Exception("Cannot compare -Inf", ErrorCodes::LOGICAL_ERROR); }
bool operator() (PositiveInfinity &) const { throw Exception("Cannot compare +Inf", ErrorCodes::LOGICAL_ERROR); }
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }

bool operator() (Array & x) const { return compareImpl<Array>(x); }
Expand Down
103 changes: 30 additions & 73 deletions src/Columns/ColumnNullable.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -546,97 +546,54 @@ namespace
{

/// The following function implements a slightly more general version
/// of getExtremes() than the implementation from ColumnVector.
/// of getExtremes() than the implementation from Not-Null IColumns.
/// It takes into account the possible presence of nullable values.
template <typename T>
void getExtremesFromNullableContent(const ColumnVector<T> & col, const NullMap & null_map, Field & min, Field & max)
void getExtremesWithNulls(const IColumn & nested_column, const NullMap & null_array, Field & min, Field & max, bool null_last = false)
{
const auto & data = col.getData();
size_t size = data.size();

if (size == 0)
{
min = Null();
max = Null();
return;
}

bool has_not_null = false;
bool has_not_nan = false;

T cur_min = 0;
T cur_max = 0;

for (size_t i = 0; i < size; ++i)
size_t number_of_nulls = 0;
size_t n = null_array.size();
NullMap not_null_array(n);
for (auto i = 0ul; i < n; ++i)
{
const T x = data[i];

if (null_map[i])
continue;

if (!has_not_null)
if (null_array[i])
{
cur_min = x;
cur_max = x;
has_not_null = true;
has_not_nan = !isNaN(x);
continue;
++number_of_nulls;
not_null_array[i] = 0;
}

if (isNaN(x))
continue;

if (!has_not_nan)
else
{
cur_min = x;
cur_max = x;
has_not_nan = true;
continue;
not_null_array[i] = 1;
}

if (x < cur_min)
cur_min = x;
else if (x > cur_max)
cur_max = x;
}

if (has_not_null)
if (number_of_nulls == 0)
{
nested_column.getExtremes(min, max);
}
else if (number_of_nulls == n)
{
min = cur_min;
max = cur_max;
min = PositiveInfinity();
max = PositiveInfinity();
}
else
{
auto filtered_column = nested_column.filter(not_null_array, -1);
filtered_column->getExtremes(min, max);
if (null_last)
max = PositiveInfinity();
}
}

}


void ColumnNullable::getExtremes(Field & min, Field & max) const
{
min = Null();
max = Null();
getExtremesWithNulls(getNestedColumn(), getNullMapData(), min, max);
}

const auto & null_map_data = getNullMapData();

if (const auto * col_i8 = typeid_cast<const ColumnInt8 *>(nested_column.get()))
getExtremesFromNullableContent<Int8>(*col_i8, null_map_data, min, max);
else if (const auto * col_i16 = typeid_cast<const ColumnInt16 *>(nested_column.get()))
getExtremesFromNullableContent<Int16>(*col_i16, null_map_data, min, max);
else if (const auto * col_i32 = typeid_cast<const ColumnInt32 *>(nested_column.get()))
getExtremesFromNullableContent<Int32>(*col_i32, null_map_data, min, max);
else if (const auto * col_i64 = typeid_cast<const ColumnInt64 *>(nested_column.get()))
getExtremesFromNullableContent<Int64>(*col_i64, null_map_data, min, max);
else if (const auto * col_u8 = typeid_cast<const ColumnUInt8 *>(nested_column.get()))
getExtremesFromNullableContent<UInt8>(*col_u8, null_map_data, min, max);
else if (const auto * col_u16 = typeid_cast<const ColumnUInt16 *>(nested_column.get()))
getExtremesFromNullableContent<UInt16>(*col_u16, null_map_data, min, max);
else if (const auto * col_u32 = typeid_cast<const ColumnUInt32 *>(nested_column.get()))
getExtremesFromNullableContent<UInt32>(*col_u32, null_map_data, min, max);
else if (const auto * col_u64 = typeid_cast<const ColumnUInt64 *>(nested_column.get()))
getExtremesFromNullableContent<UInt64>(*col_u64, null_map_data, min, max);
else if (const auto * col_f32 = typeid_cast<const ColumnFloat32 *>(nested_column.get()))
getExtremesFromNullableContent<Float32>(*col_f32, null_map_data, min, max);
else if (const auto * col_f64 = typeid_cast<const ColumnFloat64 *>(nested_column.get()))
getExtremesFromNullableContent<Float64>(*col_f64, null_map_data, min, max);
void ColumnNullable::getExtremesNullLast(Field & min, Field & max) const
{
getExtremesWithNulls(getNestedColumn(), getNullMapData(), min, max, true);
}


Expand Down
2 changes: 2 additions & 0 deletions src/Columns/ColumnNullable.h
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,8 @@ class ColumnNullable final : public COWHelper<IColumn, ColumnNullable>
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;
void getExtremes(Field & min, Field & max) const override;
// Special function for nullable minmax index
void getExtremesNullLast(Field & min, Field & max) const;

MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
{
Expand Down
10 changes: 10 additions & 0 deletions src/Common/FieldVisitorConvertToNumber.h
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,16 @@ class FieldVisitorConvertToNumber : public StaticVisitor<T>
throw Exception("Cannot convert NULL to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
}

T operator() (const NegativeInfinity &) const
{
throw Exception("Cannot convert -Inf to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
}

T operator() (const PositiveInfinity &) const
{
throw Exception("Cannot convert +Inf to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
}

T operator() (const String &) const
{
throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorDump.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
}

String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
String FieldVisitorDump::operator() (const NegativeInfinity &) const { return "-Inf"; }
String FieldVisitorDump::operator() (const PositiveInfinity &) const { return "+Inf"; }
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorDump.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ class FieldVisitorDump : public StaticVisitor<String>
{
public:
String operator() (const Null & x) const;
String operator() (const NegativeInfinity & x) const;
String operator() (const PositiveInfinity & x) const;
String operator() (const UInt64 & x) const;
String operator() (const UInt128 & x) const;
String operator() (const UInt256 & x) const;
Expand Down
12 changes: 12 additions & 0 deletions src/Common/FieldVisitorHash.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,18 @@ void FieldVisitorHash::operator() (const Null &) const
hash.update(type);
}

void FieldVisitorHash::operator() (const NegativeInfinity &) const
{
UInt8 type = Field::Types::NegativeInfinity;
hash.update(type);
}

void FieldVisitorHash::operator() (const PositiveInfinity &) const
{
UInt8 type = Field::Types::PositiveInfinity;
hash.update(type);
}

void FieldVisitorHash::operator() (const UInt64 & x) const
{
UInt8 type = Field::Types::UInt64;
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorHash.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ class FieldVisitorHash : public StaticVisitor<>
FieldVisitorHash(SipHash & hash_);

void operator() (const Null & x) const;
void operator() (const NegativeInfinity & x) const;
void operator() (const PositiveInfinity & x) const;
void operator() (const UInt64 & x) const;
void operator() (const UInt128 & x) const;
void operator() (const UInt256 & x) const;
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorSum.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
bool FieldVisitorSum::operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }

bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (NegativeInfinity &) const { throw Exception("Cannot sum -Inf", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (PositiveInfinity &) const { throw Exception("Cannot sum +Inf", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorSum.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@ class FieldVisitorSum : public StaticVisitor<bool>
bool operator() (UInt64 & x) const;
bool operator() (Float64 & x) const;
bool operator() (Null &) const;
bool operator() (NegativeInfinity & x) const;
bool operator() (PositiveInfinity & x) const;
bool operator() (String &) const;
bool operator() (Array &) const;
bool operator() (Tuple &) const;
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorToString.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@ static String formatFloat(const Float64 x)


String FieldVisitorToString::operator() (const Null &) const { return "NULL"; }
String FieldVisitorToString::operator() (const NegativeInfinity &) const { return "-Inf"; }
String FieldVisitorToString::operator() (const PositiveInfinity &) const { return "+Inf"; }
String FieldVisitorToString::operator() (const UInt64 & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorToString.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ class FieldVisitorToString : public StaticVisitor<String>
{
public:
String operator() (const Null & x) const;
String operator() (const NegativeInfinity & x) const;
String operator() (const PositiveInfinity & x) const;
String operator() (const UInt64 & x) const;
String operator() (const UInt128 & x) const;
String operator() (const UInt256 & x) const;
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorWriteBinary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ namespace DB
{

void FieldVisitorWriteBinary::operator() (const Null &, WriteBuffer &) const { }
void FieldVisitorWriteBinary::operator() (const NegativeInfinity &, WriteBuffer &) const { }
void FieldVisitorWriteBinary::operator() (const PositiveInfinity &, WriteBuffer &) const { }
void FieldVisitorWriteBinary::operator() (const UInt64 & x, WriteBuffer & buf) const { writeVarUInt(x, buf); }
void FieldVisitorWriteBinary::operator() (const Int64 & x, WriteBuffer & buf) const { writeVarInt(x, buf); }
void FieldVisitorWriteBinary::operator() (const Float64 & x, WriteBuffer & buf) const { writeFloatBinary(x, buf); }
Expand Down
2 changes: 2 additions & 0 deletions src/Common/FieldVisitorWriteBinary.h
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@ class FieldVisitorWriteBinary
{
public:
void operator() (const Null & x, WriteBuffer & buf) const;
void operator() (const NegativeInfinity & x, WriteBuffer & buf) const;
void operator() (const PositiveInfinity & x, WriteBuffer & buf) const;
void operator() (const UInt64 & x, WriteBuffer & buf) const;
void operator() (const UInt128 & x, WriteBuffer & buf) const;
void operator() (const UInt256 & x, WriteBuffer & buf) const;
Expand Down
10 changes: 9 additions & 1 deletion src/Common/FieldVisitorsAccurateComparison.h
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,12 @@ class FieldVisitorAccurateEquals : public StaticVisitor<bool>
template <typename T, typename U>
bool operator() (const T & l, const U & r) const
{
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>
|| std::is_same_v<T, NegativeInfinity> || std::is_same_v<T, PositiveInfinity>
|| std::is_same_v<U, NegativeInfinity> || std::is_same_v<U, PositiveInfinity>)
{
return std::is_same_v<T, U>;
}
else
{
if constexpr (std::is_same_v<T, U>)
Expand Down Expand Up @@ -77,6 +81,10 @@ class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
return false;
else if constexpr (std::is_same_v<T, NegativeInfinity> || std::is_same_v<U, PositiveInfinity>)
return !std::is_same_v<T, U>;
else if constexpr (std::is_same_v<U, NegativeInfinity> || std::is_same_v<T, PositiveInfinity>)
return false;
else
{
if constexpr (std::is_same_v<T, U>)
Expand Down
10 changes: 10 additions & 0 deletions src/Core/Field.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -455,6 +455,16 @@ inline void writeText(const Null &, WriteBuffer & buf)
writeText(std::string("NULL"), buf);
}

inline void writeText(const NegativeInfinity &, WriteBuffer & buf)
{
writeText(std::string("-Inf"), buf);
}

inline void writeText(const PositiveInfinity &, WriteBuffer & buf)
{
writeText(std::string("+Inf"), buf);
}

String toString(const Field & x)
{
return Field::dispatch(
Expand Down
Loading

0 comments on commit b52411a

Please sign in to comment.