Merge pull request #10623 from livace/master

Add point in polygon for non-const polygons
This commit is contained in:
alexey-milovidov 2020-05-21 15:18:24 +03:00 committed by GitHub
commit b480f137f3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 325 additions and 36 deletions

View File

@ -44,7 +44,7 @@ namespace
{
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImplWithPool(const IColumn & x, const IColumn & y, Polygon & polygon)
UInt8 callPointInPolygonImplWithPool(Float64 x, Float64 y, Polygon & polygon)
{
using Pool = ObjectPoolMap<PointInPolygonImpl, std::string>;
/// C++11 has thread-safe function-local statics on most modern compilers.
@ -63,19 +63,19 @@ ColumnPtr callPointInPolygonImplWithPool(const IColumn & x, const IColumn & y, P
std::string serialized_polygon = serialize(polygon);
auto impl = known_polygons.get(serialized_polygon, factory);
return pointInPolygon(x, y, *impl);
return impl->contains(x, y);
}
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImpl(const IColumn & x, const IColumn & y, Polygon & polygon)
UInt8 callPointInPolygonImpl(Float64 x, Float64 y, Polygon & polygon)
{
PointInPolygonImpl impl(polygon);
return pointInPolygon(x, y, impl);
return impl.contains(x, y);
}
}
template <typename PointInPolygonImpl, bool use_object_pool>
template <typename PointInConstPolygonImpl, typename PointInNonConstPolygonImpl>
class FunctionPointInPolygon : public IFunction
{
public:
@ -91,7 +91,8 @@ public:
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionPointInPolygon<PointInPolygonImpl, use_object_pool>>(context.getSettingsRef().validate_polygons);
return std::make_shared<FunctionPointInPolygon<PointInConstPolygonImpl, PointInNonConstPolygonImpl>>(
context.getSettingsRef().validate_polygons);
}
String getName() const override
@ -116,74 +117,192 @@ public:
throw Exception("Too few arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
}
auto get_message_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
auto validate_tuple = [this](size_t i, const DataTypeTuple * tuple)
{
const auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr && i != 1)
throw Exception(get_message_prefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto * tuple = checkAndGetDataType<DataTypeTuple>(array ? array->getNestedType().get() : arguments[i].get());
if (tuple == nullptr)
throw Exception(get_message_prefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(getMessagePrefix(i) + " must contain a tuple", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypes & elements = tuple->getElements();
if (elements.size() != 2)
throw Exception(get_message_prefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
throw Exception(getMessagePrefix(i) + " must have exactly two elements", ErrorCodes::BAD_ARGUMENTS);
for (auto j : ext::range(0, elements.size()))
{
if (!isNativeNumber(elements[j]))
{
throw Exception(get_message_prefix(i) + " must contains numeric tuple at position " + toString(j + 1),
throw Exception(getMessagePrefix(i) + " must contain numeric tuple at position " + toString(j + 1),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
};
validate_tuple(0, checkAndGetDataType<DataTypeTuple>(arguments[0].get()));
if (arguments.size() == 2)
{
const auto * array = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (array == nullptr)
throw Exception(getMessagePrefix(1) + " must contain an array of tuples or an array of arrays of tuples.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto * nested_array = checkAndGetDataType<DataTypeArray>(array->getNestedType().get());
if (nested_array != nullptr)
{
array = nested_array;
}
validate_tuple(1, checkAndGetDataType<DataTypeTuple>(array->getNestedType().get()));
}
else
{
for (size_t i = 1; i < arguments.size(); i++)
{
const auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr)
throw Exception(getMessagePrefix(i) + " must contain an array of tuples",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
validate_tuple(i, checkAndGetDataType<DataTypeTuple>(array->getNestedType().get()));
}
}
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IColumn * point_col = block.getByPosition(arguments[0]).column.get();
const auto * const_tuple_col = checkAndGetColumn<ColumnConst>(point_col);
if (const_tuple_col)
point_col = &const_tuple_col->getDataColumn();
const auto * tuple_col = checkAndGetColumn<ColumnTuple>(point_col);
const auto * tuple_col = checkAndGetColumn<ColumnTuple>(point_col);
if (!tuple_col)
throw Exception("First argument for function " + getName() + " must be constant array of tuples.",
ErrorCodes::ILLEGAL_COLUMN);
auto & result_column = block.safeGetByPosition(result).column;
const auto & tuple_columns = tuple_col->getColumns();
result_column = executeForType(*tuple_columns[0], *tuple_columns[1], block, arguments);
if (const_tuple_col)
const IColumn * poly_col = block.getByPosition(arguments[1]).column.get();
const auto * const_poly_col = checkAndGetColumn<ColumnConst>(poly_col);
bool point_is_const = const_tuple_col != nullptr;
bool poly_is_const = const_poly_col != nullptr;
auto call_impl = poly_is_const
? callPointInPolygonImplWithPool<Polygon, PointInConstPolygonImpl>
: callPointInPolygonImpl<Polygon, PointInNonConstPolygonImpl>;
size_t size = point_is_const && poly_is_const ? 1 : input_rows_count;
auto execution_result = ColumnVector<UInt8>::create(size);
auto & data = execution_result->getData();
Polygon polygon;
for (auto i : ext::range(0, size))
{
if (!poly_is_const || i == 0)
{
polygon = parsePolygon(block, arguments, i);
}
size_t point_index = point_is_const ? 0 : i;
data[i] = call_impl(tuple_columns[0]->getFloat64(point_index), tuple_columns[1]->getFloat64(point_index), polygon);
}
auto & result_column = block.safeGetByPosition(result).column;
result_column = std::move(execution_result);
if (point_is_const && poly_is_const)
result_column = ColumnConst::create(result_column, const_tuple_col->size());
}
private:
bool validate;
ColumnPtr executeForType(const IColumn & x, const IColumn & y, Block & block, const ColumnNumbers & arguments)
std::string getMessagePrefix(size_t i) const
{
return "Argument " + toString(i + 1) + " for function " + getName();
}
Polygon parsePolygonFromSingleColumn(Block & block, const ColumnNumbers & arguments, size_t i) const
{
const auto & poly = block.getByPosition(arguments[1]).column.get();
const auto * column_const = checkAndGetColumn<ColumnConst>(poly);
const auto * array_col =
column_const ? checkAndGetColumn<ColumnArray>(column_const->getDataColumn()) : checkAndGetColumn<ColumnArray>(poly);
if (!array_col)
throw Exception(getMessagePrefix(1) + " must contain an array of tuples or an array of arrays of tuples",
ErrorCodes::ILLEGAL_COLUMN);
const auto * nested_array_col = checkAndGetColumn<ColumnArray>(array_col->getData());
const auto & tuple_data = nested_array_col ? nested_array_col->getData() : array_col->getData();
const auto & tuple_col = checkAndGetColumn<ColumnTuple>(tuple_data);
if (!tuple_col)
throw Exception(getMessagePrefix(1) + " must contain an array of tuples or an array of arrays of tuples",
ErrorCodes::ILLEGAL_COLUMN);
const auto & tuple_columns = tuple_col->getColumns();
const auto & x_column = tuple_columns[0];
const auto & y_column = tuple_columns[1];
auto parse_polygon_part = [&x_column, &y_column](auto & container, size_t l, size_t r)
{
for (auto j : ext::range(l, r))
{
CoordinateType x_coord = x_column->getFloat64(j);
CoordinateType y_coord = y_column->getFloat64(j);
container.push_back(Point(x_coord, y_coord));
}
};
Polygon polygon;
if (nested_array_col)
{
for (auto j : ext::range(array_col->getOffsets()[i - 1], array_col->getOffsets()[i]))
{
size_t l = nested_array_col->getOffsets()[j - 1];
size_t r = nested_array_col->getOffsets()[j];
if (polygon.outer().empty())
{
parse_polygon_part(polygon.outer(), l, r);
}
else
{
polygon.inners().emplace_back();
parse_polygon_part(polygon.inners().back(), l, r);
}
}
}
else
{
size_t l = array_col->getOffsets()[i - 1];
size_t r = array_col->getOffsets()[i];
parse_polygon_part(polygon.outer(), l, r);
}
return polygon;
}
Polygon parsePolygonFromMultipleColumns(Block & block, const ColumnNumbers & arguments, size_t) const
{
Polygon polygon;
auto get_message_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
const auto * const_col = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[i]).column.get());
const auto * array_col = const_col ? checkAndGetColumn<ColumnArray>(&const_col->getDataColumn()) : nullptr;
if (!const_col)
throw Exception("Multi-argument version of function " + getName() + " works only with const polygon",
ErrorCodes::BAD_ARGUMENTS);
const auto * array_col = checkAndGetColumn<ColumnArray>(&const_col->getDataColumn());
const auto * tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
if (!tuple_col)
throw Exception(get_message_prefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception(getMessagePrefix(i) + " must be constant array of tuples", ErrorCodes::ILLEGAL_COLUMN);
const auto & tuple_columns = tuple_col->getColumns();
const auto & column_x = tuple_columns[0];
@ -197,7 +316,7 @@ private:
auto size = column_x->size();
if (size == 0)
throw Exception(get_message_prefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception(getMessagePrefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
for (auto j : ext::range(0, size))
{
@ -207,6 +326,21 @@ private:
}
}
return polygon;
}
Polygon parsePolygon(Block & block, const ColumnNumbers & arguments, size_t i) const
{
Polygon polygon;
if (arguments.size() == 2)
{
polygon = parsePolygonFromSingleColumn(block, arguments, i);
}
else
{
polygon = parsePolygonFromMultipleColumns(block, arguments, i);
}
boost::geometry::correct(polygon);
#if !defined(__clang_analyzer__) /// It does not like boost.
@ -218,19 +352,14 @@ private:
throw Exception("Polygon is not valid: " + failure_message, ErrorCodes::BAD_ARGUMENTS);
}
#endif
auto call_impl = use_object_pool
? callPointInPolygonImplWithPool<Polygon, PointInPolygonImpl>
: callPointInPolygonImpl<Polygon, PointInPolygonImpl>;
return call_impl(x, y, polygon);
return polygon;
}
};
void registerFunctionPointInPolygon(FunctionFactory & factory)
{
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid<Float64>, true>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid<Float64>, PointInPolygonTrivial<Float64>>>();
}
}

View File

@ -0,0 +1,6 @@
<test>
<create_query>CREATE TABLE point_in_polygon(`polygon` Array(Array(Float64, Float64))) ENGINE = Log()</create_query>
<create_query>insert into point_in_polygon SELECT arrayJoin(arrayMap(y -> [arrayMap(x -> (cos(x / 90. * pi()) * y, sin(x / 90. * pi()) * y), range(180))], arraySlice(range(35000), 2, 35000)))</create_query>
<query>SELECT pointInPolygon((100, 100), `polygon`) from point_in_polygon</query>
<drop_query>DROP TABLE IF EXISTS point_in_polygon</drop_query>
</test>

View File

@ -0,0 +1,68 @@
Const point; No holes
0
0
0
0
0
1
1
1
1
1
1
0
1
0
1
0
Non-const point; No holes
0
0
0
0
0
1
1
1
1
1
1
0
1
0
1
0
Const point; With holes
0
0
0
0
0
1
1
0
0
1
1
0
1
0
1
0
Non-const point; With holes
0
0
0
0
0
1
1
0
0
1
1
0
1
0
1
0

View File

@ -0,0 +1,86 @@
DROP TABLE IF EXISTS polygons;
SELECT 'Const point; No holes';
create table polygons ( id Int32, poly Array(Tuple(Int32, Int32))) engine = Log();
INSERT INTO polygons VALUES (1, [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (2, [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
SELECT pointInPolygon((-10, 0), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((0, -10), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((-5, -5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((0, 0), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((5, 5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((10, 10), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((10, 5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((5, 10), poly) FROM polygons ORDER BY id;
DROP TABLE polygons;
SELECT 'Non-const point; No holes';
create table polygons ( id Int32, pt Tuple(Int32, Int32), poly Array(Tuple(Int32, Int32))) engine = Log();
INSERT INTO polygons VALUES (1, (-10, 0), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (2, (-10, 0), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (3, (0, -10), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (4, (0, -10), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (5, (-5, -5), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (6, (-5, -5), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (7, (0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (8, (0, 0), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (9, (5, 5), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (10, (5, 5), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (11, (10, 10), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (12, (10, 10), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (13, (10, 5), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (14, (10, 5), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
INSERT INTO polygons VALUES (15, (5, 10), [(0, 0), (10, 0), (10, 10), (0, 10)]);
INSERT INTO polygons VALUES (16, (5, 10), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
SELECT pointInPolygon(pt, poly) FROM polygons ORDER BY id;
DROP TABLE polygons;
SELECT 'Const point; With holes';
create table polygons ( id Int32, poly Array(Array(Tuple(Int32, Int32)))) engine = Log();
INSERT INTO polygons VALUES (1, [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (2, [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
SELECT pointInPolygon((-10, 0), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((0, -10), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((-5, -5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((0, 0), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((5, 5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((10, 10), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((10, 5), poly) FROM polygons ORDER BY id;
SELECT pointInPolygon((5, 10), poly) FROM polygons ORDER BY id;
DROP TABLE polygons;
SELECT 'Non-const point; With holes';
create table polygons ( id Int32, pt Tuple(Int32, Int32), poly Array(Array(Tuple(Int32, Int32)))) engine = Log();
INSERT INTO polygons VALUES (1, (-10, 0), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (2, (-10, 0), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (3, (0, -10), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (4, (0, -10), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (5, (-5, -5), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (6, (-5, -5), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (7, (0, 0), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (8, (0, 0), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (9, (5, 5), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (10, (5, 5), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (11, (10, 10), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (12, (10, 10), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (13, (10, 5), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (14, (10, 5), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
INSERT INTO polygons VALUES (15, (5, 10), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
INSERT INTO polygons VALUES (16, (5, 10), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
SELECT pointInPolygon(pt, poly) FROM polygons ORDER BY id;
DROP TABLE polygons;