Fix unfinished code in GatherUtils #20272

This commit is contained in:
Alexey Milovidov 2021-04-13 16:56:39 +03:00
parent 643e697918
commit 83a78a5aa7
3 changed files with 13 additions and 41 deletions

View File

@ -82,7 +82,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericArraySlice & slice, GenericArr
sink.current_offset += slice.size;
}
else
throw Exception("Function writeSlice expect same column types for GenericArraySlice and GenericArraySink.",
throw Exception("Function writeSlice expects same column types for GenericArraySlice and GenericArraySink.",
ErrorCodes::LOGICAL_ERROR);
}
@ -162,7 +162,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericValueSlice & slice, GenericArr
++sink.current_offset;
}
else
throw Exception("Function writeSlice expect same column types for GenericValueSlice and GenericArraySink.",
throw Exception("Function writeSlice expects same column types for GenericValueSlice and GenericArraySink.",
ErrorCodes::LOGICAL_ERROR);
}
@ -609,7 +609,7 @@ bool sliceHas(const GenericArraySlice & first, const GenericArraySlice & second)
{
/// Generic arrays should have the same type in order to use column.compareAt(...)
if (!first.elements->structureEquals(*second.elements))
return false;
throw Exception("Function sliceHas expects same column types for slices.", ErrorCodes::LOGICAL_ERROR);
auto impl = sliceHasImpl<search_type, GenericArraySlice, GenericArraySlice, sliceEqualElements, insliceEqualElements>;
return impl(first, second, nullptr, nullptr);
@ -670,7 +670,7 @@ void NO_INLINE arrayAllAny(FirstSource && first, SecondSource && second, ColumnU
auto & data = result.getData();
for (auto row : ext::range(0, size))
{
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()) ? 1 : 0);
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()));
first.next();
second.next();
}

View File

@ -373,11 +373,10 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->onlyNull() && !allowArguments(array_type->getNestedType(), arguments[1]))
throw Exception("Types of array and 2nd argument of function \""
+ getName() + "\" must be identical up to nullability, cardinality, "
"numeric types, or Enum and numeric type. Passed: "
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Types of array and 2nd argument of function `{}` must be identical up to nullability, cardinality, "
"numeric types, or Enum and numeric type. Passed: {} and {}.",
getName(), arguments[0]->getName(), arguments[1]->getName());
return std::make_shared<DataTypeNumber<ResultType>>();
}

View File

@ -13,6 +13,7 @@
#include <Interpreters/castColumn.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include <ext/map.h>
namespace DB
@ -51,41 +52,13 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
size_t rows = input_rows_count;
size_t num_args = arguments.size();
DataTypePtr common_type = nullptr;
auto commonType = [&common_type, &arguments]()
{
if (common_type == nullptr)
{
DataTypes data_types;
data_types.reserve(arguments.size());
for (const auto & argument : arguments)
data_types.push_back(argument.type);
common_type = getLeastSupertype(data_types);
}
return common_type;
};
DataTypePtr common_type = getLeastSupertype(ext::map(arguments, [](auto & arg) { return arg.type; }));
Columns preprocessed_columns(num_args);
for (size_t i = 0; i < num_args; ++i)
{
const auto & argument = arguments[i];
ColumnPtr preprocessed_column = argument.column;
const auto * argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
const auto & nested_type = argument_type->getNestedType();
/// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1
if (typeid_cast<const DataTypeNothing *>(removeNullable(nested_type).get()))
preprocessed_column = castColumn(argument, commonType());
preprocessed_columns[i] = std::move(preprocessed_column);
}
preprocessed_columns[i] = castColumn(arguments[i], common_type);
std::vector<std::unique_ptr<GatherUtils::IArraySource>> sources;
@ -100,12 +73,12 @@ public:
}
if (const auto * argument_column_array = typeid_cast<const ColumnArray *>(argument_column.get()))
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows));
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, input_rows_count));
else
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
}
auto result_column = ColumnUInt8::create(rows);
auto result_column = ColumnUInt8::create(input_rows_count);
auto * result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
GatherUtils::sliceHas(*sources[0], *sources[1], search_type, *result_column_ptr);