Merge pull request #4698 from yandex/fixed-insufficient-type-check-in-has-all-any-function

Fixed insufficient type checking in hasAll, hasAny functions
This commit is contained in:
alexey-milovidov 2019-03-15 04:55:26 +03:00 committed by GitHub
commit 40944ca356
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 158 additions and 16 deletions

View File

@ -12,6 +12,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace

View File

@ -3,17 +3,12 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Core/Defines.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
}
/** A column of array values.
* In memory, it is represented as one column of a nested type, whose size is equal to the sum of the sizes of all arrays,
* and as an array of offsets in it, which allows you to get each element.
@ -121,6 +116,13 @@ public:
callback(data);
}
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnArray *>(&rhs))
return data->structureEquals(*rhs_concrete->data);
return false;
}
private:
ColumnPtr data;
ColumnPtr offsets;

View File

@ -3,6 +3,7 @@
#include <Core/Field.h>
#include <Common/Exception.h>
#include <Columns/IColumn.h>
#include <Common/typeid_cast.h>
namespace DB
@ -190,6 +191,13 @@ public:
callback(data);
}
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnConst *>(&rhs))
return data->structureEquals(*rhs_concrete->data);
return false;
}
bool onlyNull() const override { return data->isNullAt(0); }
bool isColumnConst() const override { return true; }
bool isNumeric() const override { return data->isNumeric(); }

View File

@ -2,6 +2,7 @@
#include <cmath>
#include <Common/typeid_cast.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVectorHelper.h>
@ -133,6 +134,13 @@ public:
void gather(ColumnGathererStream & gatherer_stream) override;
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnDecimal<T> *>(&rhs))
return scale == rhs_concrete->scale;
return false;
}
void insert(const T value) { data.push_back(value); }
Container & getData() { return data; }

View File

@ -2,6 +2,7 @@
#include <Common/PODArray.h>
#include <Common/memcmpSmall.h>
#include <Common/typeid_cast.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVectorHelper.h>
@ -134,6 +135,12 @@ public:
void getExtremes(Field & min, Field & max) const override;
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnFixedString *>(&rhs))
return n == rhs_concrete->n;
return false;
}
bool canBeInsideNullable() const override { return true; }

View File

@ -5,6 +5,7 @@
#include <AggregateFunctions/AggregateFunctionCount.h>
#include "ColumnsNumber.h"
namespace DB
{
@ -132,6 +133,14 @@ public:
callback(dictionary.getColumnUniquePtr());
}
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_low_cardinality = typeid_cast<const ColumnLowCardinality *>(&rhs))
return idx.getPositions()->structureEquals(*rhs_low_cardinality->idx.getPositions())
&& dictionary.getColumnUnique().structureEquals(rhs_low_cardinality->dictionary.getColumnUnique());
return false;
}
bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); }
bool isFixedAndContiguous() const override { return false; }
size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); }

View File

@ -23,6 +23,11 @@ public:
MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnNothing::create(s_); }
bool canBeInsideNullable() const override { return true; }
bool structureEquals(const IColumn & rhs) const override
{
return typeid(rhs) == typeid(ColumnNothing);
}
};
}

View File

@ -2,6 +2,8 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnsNumber.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -89,6 +91,13 @@ public:
callback(null_map);
}
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_nullable = typeid_cast<const ColumnNullable *>(&rhs))
return nested_column->structureEquals(*rhs_nullable->nested_column);
return false;
}
bool isColumnNullable() const override { return true; }
bool isFixedAndContiguous() const override { return false; }
bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); }

View File

@ -231,6 +231,11 @@ public:
bool canBeInsideNullable() const override { return true; }
bool structureEquals(const IColumn & rhs) const override
{
return typeid(rhs) == typeid(ColumnString);
}
Chars & getChars() { return chars; }
const Chars & getChars() const { return chars; }

View File

@ -4,6 +4,7 @@
#include <IO/Operators.h>
#include <ext/map.h>
#include <ext/range.h>
#include <Common/typeid_cast.h>
namespace DB
@ -341,6 +342,23 @@ void ColumnTuple::forEachSubcolumn(ColumnCallback callback)
callback(column);
}
bool ColumnTuple::structureEquals(const IColumn & rhs) const
{
if (auto rhs_tuple = typeid_cast<const ColumnTuple *>(&rhs))
{
const size_t tuple_size = columns.size();
if (tuple_size != rhs_tuple->columns.size())
return false;
for (const auto i : ext::range(0, tuple_size))
if (!columns[i]->structureEquals(*rhs_tuple->columns[i]))
return false;
return true;
}
else
return false;
}
}

View File

@ -73,6 +73,7 @@ public:
size_t allocatedBytes() const override;
void protect() override;
void forEachSubcolumn(ColumnCallback callback) override;
bool structureEquals(const IColumn & rhs) const override;
size_t tupleSize() const { return columns.size(); }

View File

@ -95,6 +95,13 @@ public:
nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask);
}
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnUnique *>(&rhs))
return column_holder->structureEquals(*rhs_concrete->column_holder);
return false;
}
const UInt64 * tryGetSavedHash() const override { return index.tryGetSavedHash(); }
UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); }

View File

@ -251,6 +251,12 @@ public:
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
bool structureEquals(const IColumn & rhs) const override
{
return typeid(rhs) == typeid(ColumnVector<T>);
}
/** More efficient methods of manipulation - to manipulate with data directly. */
Container & getData()
{

View File

@ -262,6 +262,13 @@ public:
using ColumnCallback = std::function<void(Ptr&)>;
virtual void forEachSubcolumn(ColumnCallback) {}
/// Columns have equal structure.
/// If true - you can use "compareAt", "insertFrom", etc. methods.
virtual bool structureEquals(const IColumn &) const
{
throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
MutablePtr mutate() const &&
{

View File

@ -33,6 +33,7 @@ namespace ErrorCodes
{
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
}
@ -123,8 +124,8 @@ public:
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -36,6 +36,7 @@ namespace ErrorCodes
{
extern const int DICTIONARIES_WAS_NOT_LOADED;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}

View File

@ -44,6 +44,8 @@ namespace ErrorCodes
extern const int UNKNOWN_TYPE;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TYPE_MISMATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
}
/** Functions that use plug-ins (external) dictionaries.

View File

@ -20,6 +20,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
}
enum ClusterOperation

View File

@ -48,6 +48,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_COLUMN;
}

View File

@ -21,6 +21,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}

View File

@ -38,6 +38,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct HasParam
{
using ResultType = UInt8;

View File

@ -53,7 +53,7 @@ inline ALWAYS_INLINE void writeSlice(const StringSource::Slice & slice, FixedStr
/// Assuming same types of underlying columns for slice and sink if (ArraySlice, ArraySink) is (GenericArraySlice, GenericArraySink).
inline ALWAYS_INLINE void writeSlice(const GenericArraySlice & slice, GenericArraySink & sink)
{
if (typeid(slice.elements) == typeid(static_cast<const IColumn *>(&sink.elements)))
if (slice.elements->structureEquals(sink.elements))
{
sink.elements.insertRangeFrom(*slice.elements, slice.begin, slice.size);
sink.current_offset += slice.size;
@ -125,7 +125,7 @@ void writeSlice(const NumericValueSlice<T> & slice, NumericArraySink<U> & sink)
/// Assuming same types of underlying columns for slice and sink if (ArraySlice, ArraySink) is (GenericValueSlice, GenericArraySink).
inline ALWAYS_INLINE void writeSlice(const GenericValueSlice & slice, GenericArraySink & sink)
{
if (typeid(slice.elements) == typeid(static_cast<const IColumn *>(&sink.elements)))
if (slice.elements->structureEquals(sink.elements))
{
sink.elements.insertFrom(*slice.elements, slice.position);
++sink.current_offset;
@ -457,7 +457,7 @@ template <bool all>
bool sliceHas(const GenericArraySlice & first, const GenericArraySlice & second)
{
/// Generic arrays should have the same type in order to use column.compareAt(...)
if (typeid(*first.elements) != typeid(*second.elements))
if (!first.elements->structureEquals(*second.elements))
return false;
auto impl = sliceHasImpl<all, GenericArraySlice, GenericArraySlice, sliceEqualElements>;

View File

@ -14,7 +14,15 @@
#include <Functions/GatherUtils/Slices.h>
#include <Functions/FunctionHelpers.h>
namespace DB::GatherUtils
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
namespace GatherUtils
{
template <typename T>
@ -660,3 +668,5 @@ struct NullableValueSource : public ValueSource
};
}
}

View File

@ -6,6 +6,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments)
{
const size_t num_arguments = arguments.size();

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}

View File

@ -7,6 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/// flatten([[1, 2, 3], [4, 5]]) = [1, 2, 3, 4, 5] - flatten array.
class FunctionFlatten : public IFunction
{

View File

@ -53,10 +53,8 @@ public:
size_t rows = input_rows_count;
size_t num_args = arguments.size();
auto result_column = ColumnUInt8::create(rows);
DataTypePtr common_type = nullptr;
auto commonType = [& common_type, & block, & arguments]()
auto commonType = [&common_type, &block, &arguments]()
{
if (common_type == nullptr)
{
@ -106,6 +104,7 @@ public:
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
}
auto result_column = ColumnUInt8::create(rows);
auto result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
GatherUtils::sliceHas(*sources[0], *sources[1], all, *result_column_ptr);

View File

@ -8,6 +8,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Creates an array, multiplying the column (the first argument) by the number of elements in the array (the second argument).
*/
class FunctionReplicate : public IFunction

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** timeSlots(StartTime, Duration)

View File

@ -11,6 +11,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{

View File

@ -0,0 +1,3 @@
SELECT hasAny([['Hello, world']], [[[]]]);
SELECT hasAny([['Hello, world']], [['Hello', 'world'], ['Hello, world']]);
SELECT hasAll([['Hello, world']], [['Hello', 'world'], ['Hello, world']]);