mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #11144 from ClickHouse/dont-validate-polygons-in-non-constant-case
Fix performance issue with pointInPolygon
This commit is contained in:
commit
08c7277d2b
@ -6,6 +6,7 @@
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
/// Warning in boost::geometry during template strategy substitution.
|
||||
@ -31,7 +32,7 @@
|
||||
#include <iterator>
|
||||
#include <cmath>
|
||||
#include <algorithm>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -615,7 +616,7 @@ struct CallPointInPolygon<>
|
||||
};
|
||||
|
||||
template <typename PointInPolygonImpl>
|
||||
ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
|
||||
NO_INLINE ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
|
||||
{
|
||||
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNativeNumbers>::Type;
|
||||
return Impl::call(x, y, impl);
|
||||
@ -623,30 +624,27 @@ ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImp
|
||||
|
||||
|
||||
template <typename Polygon>
|
||||
std::string serialize(Polygon && polygon)
|
||||
UInt128 sipHash128(Polygon && polygon)
|
||||
{
|
||||
WriteBufferFromOwnString buffer;
|
||||
SipHash hash;
|
||||
|
||||
using RingType = typename std::decay_t<Polygon>::ring_type;
|
||||
|
||||
auto serializeRing = [&buffer](const RingType & ring)
|
||||
auto hash_ring = [&hash](const auto & ring)
|
||||
{
|
||||
writeBinary(ring.size(), buffer);
|
||||
for (const auto & point : ring)
|
||||
{
|
||||
writeBinary(point.x(), buffer);
|
||||
writeBinary(point.y(), buffer);
|
||||
}
|
||||
UInt32 size = ring.size();
|
||||
hash.update(size);
|
||||
hash.update(reinterpret_cast<const char *>(ring.data()), size * sizeof(ring[0]));
|
||||
};
|
||||
|
||||
serializeRing(polygon.outer());
|
||||
hash_ring(polygon.outer());
|
||||
|
||||
const auto & inners = polygon.inners();
|
||||
writeBinary(inners.size(), buffer);
|
||||
hash.update(inners.size());
|
||||
for (auto & inner : inners)
|
||||
serializeRing(inner);
|
||||
hash_ring(inner);
|
||||
|
||||
return buffer.str();
|
||||
UInt128 res;
|
||||
hash.get128(res.low, res.high);
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/ObjectPool.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
@ -19,6 +20,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <string>
|
||||
#include <memory>
|
||||
@ -41,58 +43,24 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename Polygon, typename PointInPolygonImpl>
|
||||
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.
|
||||
static Pool known_polygons;
|
||||
using CoordinateType = Float64;
|
||||
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
|
||||
using Polygon = boost::geometry::model::polygon<Point, false>;
|
||||
using Box = boost::geometry::model::box<Point>;
|
||||
|
||||
auto factory = [& polygon]()
|
||||
{
|
||||
auto ptr = std::make_unique<PointInPolygonImpl>(polygon);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::PolygonsAddedToPool);
|
||||
ProfileEvents::increment(ProfileEvents::PolygonsInPoolAllocatedBytes, ptr->getAllocatedBytes());
|
||||
|
||||
return ptr.release();
|
||||
};
|
||||
|
||||
std::string serialized_polygon = serialize(polygon);
|
||||
auto impl = known_polygons.get(serialized_polygon, factory);
|
||||
|
||||
return impl->contains(x, y);
|
||||
}
|
||||
|
||||
template <typename Polygon, typename PointInPolygonImpl>
|
||||
UInt8 callPointInPolygonImpl(Float64 x, Float64 y, Polygon & polygon)
|
||||
{
|
||||
PointInPolygonImpl impl(polygon);
|
||||
return impl.contains(x, y);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
template <typename PointInConstPolygonImpl, typename PointInNonConstPolygonImpl>
|
||||
template <typename PointInConstPolygonImpl>
|
||||
class FunctionPointInPolygon : public IFunction
|
||||
{
|
||||
public:
|
||||
using CoordinateType = Float64;
|
||||
|
||||
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
|
||||
using Polygon = boost::geometry::model::polygon<Point, false>;
|
||||
using Box = boost::geometry::model::box<Point>;
|
||||
|
||||
static inline const char * name = "pointInPolygon";
|
||||
|
||||
explicit FunctionPointInPolygon(bool validate_) : validate(validate_) {}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionPointInPolygon<PointInConstPolygonImpl, PointInNonConstPolygonImpl>>(
|
||||
return std::make_shared<FunctionPointInPolygon<PointInConstPolygonImpl>>(
|
||||
context.getSettingsRef().validate_polygons);
|
||||
}
|
||||
|
||||
@ -118,6 +86,16 @@ public:
|
||||
throw Exception("Too few arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
|
||||
}
|
||||
|
||||
/** We allow function invocation in one of the following forms:
|
||||
*
|
||||
* pointInPolygon((x, y), [(x1, y1), (x2, y2), ...])
|
||||
* - simple polygon
|
||||
* pointInPolygon((x, y), [(x1, y1), (x2, y2), ...], [(x21, y21), (x22, y22), ...], ...)
|
||||
* - polygon with a number of holes, each hole as a subsequent argument.
|
||||
* pointInPolygon((x, y), [[(x1, y1), (x2, y2), ...], [(x21, y21), (x22, y22), ...], ...])
|
||||
* - polygon with a number of holes, all as multidimensional array
|
||||
*/
|
||||
|
||||
auto validate_tuple = [this](size_t i, const DataTypeTuple * tuple)
|
||||
{
|
||||
if (tuple == nullptr)
|
||||
@ -185,39 +163,115 @@ public:
|
||||
|
||||
const auto & tuple_columns = tuple_col->getColumns();
|
||||
|
||||
const IColumn * poly_col = block.getByPosition(arguments[1]).column.get();
|
||||
const auto * const_poly_col = checkAndGetColumn<ColumnConst>(poly_col);
|
||||
const ColumnWithTypeAndName poly = block.getByPosition(arguments[1]);
|
||||
const IColumn * poly_col = poly.column.get();
|
||||
const ColumnConst * 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>;
|
||||
/// Two different algorithms are used for constant and non constant polygons.
|
||||
/// Constant polygons are preprocessed to speed up matching.
|
||||
/// For non-constant polygons, we cannot spend time for preprocessing
|
||||
/// and have to quickly match on the fly without creating temporary data structures.
|
||||
|
||||
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)
|
||||
{
|
||||
if (!poly_is_const || i == 0)
|
||||
Polygon polygon;
|
||||
parseConstPolygon(block, arguments, polygon);
|
||||
|
||||
/// Polygons are preprocessed and saved in cache.
|
||||
/// Preprocessing can be computationally heavy but dramatically speeds up matching.
|
||||
|
||||
using Pool = ObjectPoolMap<PointInConstPolygonImpl, UInt128>;
|
||||
/// C++11 has thread-safe function-local statics.
|
||||
static Pool known_polygons;
|
||||
|
||||
auto factory = [&polygon]()
|
||||
{
|
||||
polygon = parsePolygon(block, arguments, i);
|
||||
auto ptr = std::make_unique<PointInConstPolygonImpl>(polygon);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::PolygonsAddedToPool);
|
||||
ProfileEvents::increment(ProfileEvents::PolygonsInPoolAllocatedBytes, ptr->getAllocatedBytes());
|
||||
|
||||
return ptr.release();
|
||||
};
|
||||
|
||||
auto impl = known_polygons.get(sipHash128(polygon), factory);
|
||||
|
||||
if (point_is_const)
|
||||
{
|
||||
bool is_in = impl->contains(tuple_columns[0]->getFloat64(0), tuple_columns[1]->getFloat64(0));
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, is_in);
|
||||
}
|
||||
else
|
||||
{
|
||||
block.getByPosition(result).column = pointInPolygon(*tuple_columns[0], *tuple_columns[1], *impl);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Multi-argument version of function " + getName() + " works only with const polygon",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto res_column = ColumnVector<UInt8>::create(input_rows_count);
|
||||
auto & data = res_column->getData();
|
||||
|
||||
/// A polygon, possibly with holes, is represented by 2d array:
|
||||
/// [[(outer_x_1, outer_y_1, ...)], [(hole1_x_1, hole1_y_1), ...], ...]
|
||||
///
|
||||
/// Or, a polygon without holes can be represented by 1d array:
|
||||
/// [(outer_x_1, outer_y_1, ...)]
|
||||
|
||||
if (isTwoDimensionalArray(*block.getByPosition(arguments[1]).type))
|
||||
{
|
||||
/// We cast everything to Float64 in advance (in batch fashion)
|
||||
/// to avoid casting with virtual calls in a loop.
|
||||
/// Note that if the type is already Float64, the operation in noop.
|
||||
|
||||
ColumnPtr polygon_column_float64 = castColumn(
|
||||
block.getByPosition(arguments[1]),
|
||||
std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{
|
||||
std::make_shared<DataTypeFloat64>(),
|
||||
std::make_shared<DataTypeFloat64>()}))));
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
size_t point_index = point_is_const ? 0 : i;
|
||||
data[i] = isInsidePolygonWithHoles(
|
||||
tuple_columns[0]->getFloat64(point_index),
|
||||
tuple_columns[1]->getFloat64(point_index),
|
||||
*polygon_column_float64,
|
||||
i);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnPtr polygon_column_float64 = castColumn(
|
||||
block.getByPosition(arguments[1]),
|
||||
std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{
|
||||
std::make_shared<DataTypeFloat64>(),
|
||||
std::make_shared<DataTypeFloat64>()})));
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
size_t point_index = point_is_const ? 0 : i;
|
||||
data[i] = isInsidePolygonWithoutHoles(
|
||||
tuple_columns[0]->getFloat64(point_index),
|
||||
tuple_columns[1]->getFloat64(point_index),
|
||||
*polygon_column_float64,
|
||||
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);
|
||||
block.getByPosition(result).column = std::move(res_column);
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
@ -226,78 +280,202 @@ private:
|
||||
return "Argument " + toString(i + 1) + " for function " + getName();
|
||||
}
|
||||
|
||||
Polygon parsePolygonFromSingleColumn(Block & block, const ColumnNumbers & arguments, size_t i) const
|
||||
bool isTwoDimensionalArray(const IDataType & type) 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;
|
||||
return WhichDataType(type).isArray()
|
||||
&& WhichDataType(static_cast<const DataTypeArray &>(type).getNestedType()).isArray();
|
||||
}
|
||||
|
||||
Polygon parsePolygonFromMultipleColumns(Block & block, const ColumnNumbers & arguments, size_t) const
|
||||
{
|
||||
Polygon polygon;
|
||||
/// Implementation methods to check point-in-polygon on the fly (for non-const polygons).
|
||||
|
||||
bool isInsideRing(
|
||||
Float64 point_x,
|
||||
Float64 point_y,
|
||||
const Float64 * ring_x_data,
|
||||
const Float64 * ring_y_data,
|
||||
size_t ring_begin,
|
||||
size_t ring_end) const
|
||||
{
|
||||
size_t size = ring_end - ring_begin;
|
||||
|
||||
if (size < 2)
|
||||
return false;
|
||||
|
||||
/** This is the algorithm by W. Randolph Franklin
|
||||
* https://wrf.ecse.rpi.edu//Research/Short_Notes/pnpoly.html
|
||||
*
|
||||
* Basically it works like this:
|
||||
* From the point, cast a horizontal ray to the right
|
||||
* and count the number of intersections with polygon edges
|
||||
* (every edge is considered semi-closed, e.g. includes the first vertex and does not include the last)
|
||||
*
|
||||
* Advantages:
|
||||
* - works regardless to the orientation;
|
||||
* - for polygon without holes:
|
||||
* works regardless to whether the polygon is closed by last vertex equals to first vertex or not;
|
||||
* (no need to preprocess polygon in any way)
|
||||
* - easy to apply for polygons with holes and for multi-polygons;
|
||||
* - it even works for polygons with self-intersections in a reasonable way;
|
||||
* - simplicity and performance;
|
||||
* - can be additionally speed up with loop unrolling and/or binary search for possible intersecting edges.
|
||||
*
|
||||
* Drawbacks:
|
||||
* - it's unspecified whether a point of the edge is inside or outside of a polygon
|
||||
* (looks like it's inside for "left" edges and outside for "right" edges)
|
||||
*
|
||||
* Why not to apply the same algorithm available in boost::geometry?
|
||||
* It will require to move data from columns to temporary containers.
|
||||
* Despite the fact that the boost library is template based and allows arbitrary containers and points,
|
||||
* it's diffucult to use without data movement because
|
||||
* we use structure-of-arrays for coordinates instead of arrays-of-structures.
|
||||
*/
|
||||
|
||||
size_t vertex1_idx = ring_begin;
|
||||
size_t vertex2_idx = ring_end - 1;
|
||||
bool res = false;
|
||||
|
||||
while (vertex1_idx < ring_end)
|
||||
{
|
||||
/// First condition checks that the point is inside horizontal row between edge top and bottom y-coordinate.
|
||||
/// Second condition checks for intersection with the edge.
|
||||
|
||||
if (((ring_y_data[vertex1_idx] > point_y) != (ring_y_data[vertex2_idx] > point_y))
|
||||
&& (point_x < (ring_x_data[vertex2_idx] - ring_x_data[vertex1_idx])
|
||||
* (point_y - ring_y_data[vertex1_idx]) / (ring_y_data[vertex2_idx] - ring_y_data[vertex1_idx])
|
||||
+ ring_x_data[vertex1_idx]))
|
||||
{
|
||||
res = !res;
|
||||
}
|
||||
|
||||
vertex2_idx = vertex1_idx;
|
||||
++vertex1_idx;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
bool isInsidePolygonWithoutHoles(
|
||||
Float64 point_x,
|
||||
Float64 point_y,
|
||||
const IColumn & polygon_column,
|
||||
size_t i) const
|
||||
{
|
||||
const auto & array_col = static_cast<const ColumnArray &>(polygon_column);
|
||||
|
||||
size_t begin = array_col.getOffsets()[i - 1];
|
||||
size_t end = array_col.getOffsets()[i];
|
||||
size_t size = end - begin;
|
||||
|
||||
if (size < 2)
|
||||
return false;
|
||||
|
||||
const auto & tuple_columns = static_cast<const ColumnTuple &>(array_col.getData()).getColumns();
|
||||
const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
|
||||
const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
|
||||
|
||||
return isInsideRing(point_x, point_y, x_data, y_data, begin, end);
|
||||
}
|
||||
|
||||
bool isInsidePolygonWithHoles(
|
||||
Float64 point_x,
|
||||
Float64 point_y,
|
||||
const IColumn & polygon_column,
|
||||
size_t i) const
|
||||
{
|
||||
const auto & array_col = static_cast<const ColumnArray &>(polygon_column);
|
||||
size_t rings_begin = array_col.getOffsets()[i - 1];
|
||||
size_t rings_end = array_col.getOffsets()[i];
|
||||
|
||||
const auto & nested_array_col = static_cast<const ColumnArray &>(array_col.getData());
|
||||
const auto & tuple_columns = static_cast<const ColumnTuple &>(nested_array_col.getData()).getColumns();
|
||||
const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
|
||||
const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
|
||||
|
||||
for (size_t j = rings_begin; j < rings_end; ++j)
|
||||
{
|
||||
size_t begin = nested_array_col.getOffsets()[j - 1];
|
||||
size_t end = nested_array_col.getOffsets()[j];
|
||||
|
||||
if (j == rings_begin)
|
||||
{
|
||||
if (!isInsideRing(point_x, point_y, x_data, y_data, begin, end))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (isInsideRing(point_x, point_y, x_data, y_data, begin, end))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Implementation methods to create boost::geometry::polygon for subsequent preprocessing.
|
||||
/// They are used to optimize matching for constant polygons. Preprocessing may take significant amount of time.
|
||||
|
||||
template <typename T>
|
||||
void parseRing(
|
||||
const Float64 * x_data,
|
||||
const Float64 * y_data,
|
||||
size_t begin,
|
||||
size_t end,
|
||||
T & out_container) const
|
||||
{
|
||||
out_container.reserve(end - begin);
|
||||
for (size_t i = begin; i < end; ++i)
|
||||
out_container.emplace_back(x_data[i], y_data[i]);
|
||||
}
|
||||
|
||||
void parseConstPolygonWithoutHolesFromSingleColumn(const IColumn & column, size_t i, Polygon & out_polygon) const
|
||||
{
|
||||
const auto & array_col = static_cast<const ColumnArray &>(column);
|
||||
size_t begin = array_col.getOffsets()[i - 1];
|
||||
size_t end = array_col.getOffsets()[i];
|
||||
|
||||
const auto & tuple_columns = static_cast<const ColumnTuple &>(array_col.getData()).getColumns();
|
||||
const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
|
||||
const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
|
||||
|
||||
parseRing(x_data, y_data, begin, end, out_polygon.outer());
|
||||
}
|
||||
|
||||
void parseConstPolygonWithHolesFromSingleColumn(const IColumn & column, size_t i, Polygon & out_polygon) const
|
||||
{
|
||||
const auto & array_col = static_cast<const ColumnArray &>(column);
|
||||
size_t rings_begin = array_col.getOffsets()[i - 1];
|
||||
size_t rings_end = array_col.getOffsets()[i];
|
||||
|
||||
const auto & nested_array_col = static_cast<const ColumnArray &>(array_col.getData());
|
||||
const auto & tuple_columns = static_cast<const ColumnTuple &>(nested_array_col.getData()).getColumns();
|
||||
const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
|
||||
const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
|
||||
|
||||
for (size_t j = rings_begin; j < rings_end; ++j)
|
||||
{
|
||||
size_t begin = nested_array_col.getOffsets()[j - 1];
|
||||
size_t end = nested_array_col.getOffsets()[j];
|
||||
|
||||
if (out_polygon.outer().empty())
|
||||
{
|
||||
parseRing(x_data, y_data, begin, end, out_polygon.outer());
|
||||
}
|
||||
else
|
||||
{
|
||||
out_polygon.inners().emplace_back();
|
||||
parseRing(x_data, y_data, begin, end, out_polygon.inners().back());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void parseConstPolygonWithHolesFromMultipleColumns(Block & block, const ColumnNumbers & arguments, Polygon & out_polygon) const
|
||||
{
|
||||
for (size_t i = 1; i < arguments.size(); ++i)
|
||||
{
|
||||
const auto * const_col = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[i]).column.get());
|
||||
if (!const_col)
|
||||
throw Exception("Multi-argument version of function " + getName() + " works only with const polygon",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto * array_col = checkAndGetColumn<ColumnArray>(&const_col->getDataColumn());
|
||||
const auto * tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
|
||||
@ -309,10 +487,10 @@ private:
|
||||
const auto & column_x = tuple_columns[0];
|
||||
const auto & column_y = tuple_columns[1];
|
||||
|
||||
if (!polygon.outer().empty())
|
||||
polygon.inners().emplace_back();
|
||||
if (!out_polygon.outer().empty())
|
||||
out_polygon.inners().emplace_back();
|
||||
|
||||
auto & container = polygon.outer().empty() ? polygon.outer() : polygon.inners().back();
|
||||
auto & container = out_polygon.outer().empty() ? out_polygon.outer() : out_polygon.inners().back();
|
||||
|
||||
auto size = column_x->size();
|
||||
|
||||
@ -326,41 +504,52 @@ private:
|
||||
container.push_back(Point(x_coord, y_coord));
|
||||
}
|
||||
}
|
||||
|
||||
return polygon;
|
||||
}
|
||||
|
||||
Polygon parsePolygon(Block & block, const ColumnNumbers & arguments, size_t i) const
|
||||
void parseConstPolygonFromSingleColumn(Block & block, const ColumnNumbers & arguments, Polygon & out_polygon) const
|
||||
{
|
||||
Polygon polygon;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
polygon = parsePolygonFromSingleColumn(block, arguments, i);
|
||||
}
|
||||
else
|
||||
{
|
||||
polygon = parsePolygonFromMultipleColumns(block, arguments, i);
|
||||
}
|
||||
ColumnPtr polygon_column_float64 = castColumn(
|
||||
block.getByPosition(arguments[1]),
|
||||
std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{
|
||||
std::make_shared<DataTypeFloat64>(),
|
||||
std::make_shared<DataTypeFloat64>()})));
|
||||
|
||||
boost::geometry::correct(polygon);
|
||||
const ColumnConst & column_const = typeid_cast<const ColumnConst &>(*polygon_column_float64);
|
||||
const IColumn & column_const_data = column_const.getDataColumn();
|
||||
|
||||
if (isTwoDimensionalArray(*block.getByPosition(arguments[1]).type))
|
||||
parseConstPolygonWithHolesFromSingleColumn(column_const_data, 0, out_polygon);
|
||||
else
|
||||
parseConstPolygonWithoutHolesFromSingleColumn(column_const_data, 0, out_polygon);
|
||||
}
|
||||
|
||||
void parseConstPolygon(Block & block, const ColumnNumbers & arguments, Polygon & out_polygon) const
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
parseConstPolygonFromSingleColumn(block, arguments, out_polygon);
|
||||
else
|
||||
parseConstPolygonWithHolesFromMultipleColumns(block, arguments, out_polygon);
|
||||
|
||||
/// Fix orientation and close rings. It's required for subsequent processing.
|
||||
boost::geometry::correct(out_polygon);
|
||||
|
||||
#if !defined(__clang_analyzer__) /// It does not like boost.
|
||||
if (validate)
|
||||
{
|
||||
std::string failure_message;
|
||||
auto is_valid = boost::geometry::is_valid(polygon, failure_message);
|
||||
auto is_valid = boost::geometry::is_valid(out_polygon, failure_message);
|
||||
if (!is_valid)
|
||||
throw Exception("Polygon is not valid: " + failure_message, ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
#endif
|
||||
return polygon;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionPointInPolygon(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid<Float64>, PointInPolygonTrivial<Float64>>>();
|
||||
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid<Float64>>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,7 +4,7 @@
|
||||
INSERT INTO polygons
|
||||
WITH number + 1 AS radius
|
||||
SELECT [arrayMap(x -> (cos(x / 90. * pi()) * radius, sin(x / 90. * pi()) * radius), range(180))]
|
||||
FROM numbers(1000)
|
||||
FROM numbers(100000)
|
||||
</create_query>
|
||||
|
||||
<query>SELECT pointInPolygon((100, 100), polygon) FROM polygons</query>
|
||||
|
7
tests/performance/point_in_polygon_const.xml
Normal file
7
tests/performance/point_in_polygon_const.xml
Normal file
File diff suppressed because one or more lines are too long
@ -6,14 +6,14 @@ 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((-9, 0), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((0, -9), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((-4, -4), 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;
|
||||
SELECT pointInPolygon((4, 4), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((9, 9), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((9, 4), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((4, 9), poly) FROM polygons ORDER BY id;
|
||||
|
||||
DROP TABLE polygons;
|
||||
|
||||
@ -21,22 +21,22 @@ 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 (1, (-9, 0), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (2, (-9, 0), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
INSERT INTO polygons VALUES (3, (0, -9), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (4, (0, -9), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
INSERT INTO polygons VALUES (5, (-4, -4), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (6, (-4, -4), [(-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)]);
|
||||
INSERT INTO polygons VALUES (9, (4, 4), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (10, (4, 4), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
INSERT INTO polygons VALUES (11, (9, 9), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (12, (9, 9), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
INSERT INTO polygons VALUES (13, (9, 4), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (14, (9, 4), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
INSERT INTO polygons VALUES (15, (4, 9), [(0, 0), (10, 0), (10, 10), (0, 10)]);
|
||||
INSERT INTO polygons VALUES (16, (4, 9), [(-5, -5), (5, -5), (5, 5), (-5, 5)]);
|
||||
|
||||
SELECT pointInPolygon(pt, poly) FROM polygons ORDER BY id;
|
||||
|
||||
@ -49,14 +49,14 @@ create table polygons ( id Int32, poly Array(Array(Tuple(Int32, Int32)))) engine
|
||||
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((-9, 0), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((0, -9), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((-4, -4), 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;
|
||||
SELECT pointInPolygon((4, 4), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((9, 9), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((9, 4), poly) FROM polygons ORDER BY id;
|
||||
SELECT pointInPolygon((4, 9), poly) FROM polygons ORDER BY id;
|
||||
|
||||
DROP TABLE polygons;
|
||||
|
||||
@ -64,23 +64,23 @@ 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 (1, (-9, 0), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (2, (-9, 0), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
|
||||
INSERT INTO polygons VALUES (3, (0, -9), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (4, (0, -9), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
|
||||
INSERT INTO polygons VALUES (5, (-4, -4), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (6, (-4, -4), [[(-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)]]);
|
||||
INSERT INTO polygons VALUES (9, (4, 4), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (10, (4, 4), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
|
||||
INSERT INTO polygons VALUES (11, (9, 9), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (12, (9, 9), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
|
||||
INSERT INTO polygons VALUES (13, (9, 4), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (14, (9, 4), [[(-5, -5), (5, -5), (5, 5), (-5, 5)], [(-1, -1), (1, -1), (1, 1), (-1, 1)]]);
|
||||
INSERT INTO polygons VALUES (15, (4, 9), [[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (6, 4), (6, 6), (4, 6)]]);
|
||||
INSERT INTO polygons VALUES (16, (4, 9), [[(-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;
|
||||
DROP TABLE polygons;
|
||||
|
Loading…
Reference in New Issue
Block a user