Point in polygon (#1264)

* added FunctionPointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix PointInPolygonWithGrid [#CLICKHOUSE-3201]

* added test for PointInPolygonWithGrid [#CLICKHOUSE-3201]

* added test for PointInPolygonWithGrid with holes [#CLICKHOUSE-3201]

* added arbitrary orientation for polygons in PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix arbitrary orientation for polygons in PointInPolygonWithGrid [#CLICKHOUSE-3201]

* added test for arbitrary orientation for polygons in PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fixed shifted grid for PointInPolygonWithGrid, added test [CLICKHOUSE-3201]

* expand box for complex polygons in PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fix expand box for complex polygons in PointInPolygonWithGrid [#CLICKHOUSE-3201]

* added pool to PointInPolygonWithGrid [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* changed FunctionPointInPolygonWithGrid signature [#CLICKHOUSE-3201]

* changed FunctionPointInPolygonWithGrid signature [#CLICKHOUSE-3201]

* fixed test [#CLICKHOUSE-3201]

* unified signature for PointInPolygon [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* supported Float64 for pointInPolygon [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fix build [#CLICKHOUSE-3201]

* fixed check for numeric tuple in PointInPolygon function [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* added profile events polygons pool [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* fixed build [#CLICKHOUSE-3201]

* Update FunctionsGeo.cpp
This commit is contained in:
KochetovNicolai 2017-09-20 05:30:44 +03:00 committed by alexey-milovidov
parent 22e89f8e69
commit e0efb7e8c1
5 changed files with 971 additions and 108 deletions

View File

@ -124,8 +124,9 @@
M(DictCacheLockReadNs) \
\
M(DistributedSyncInsertionTimeoutExceeded) \
M(DataAfterMergeDiffersFromReplica)
M(DataAfterMergeDiffersFromReplica) \
M(PolygonsAddedToPool) \
M(PolygonsInPoolAllocatedBytes) \
namespace ProfileEvents
{

View File

@ -1,5 +1,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsGeo.h>
#include <Functions/GeoUtils.h>
#include <Functions/ObjectPool.h>
#include <boost/geometry.hpp>
#include <boost/geometry/geometries/point_xy.hpp>
@ -9,36 +11,79 @@
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnTuple.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Common/ProfileEvents.h>
namespace ProfileEvents
{
extern const Event PolygonsAddedToPool;
extern const Event PolygonsInPoolAllocatedBytes;
}
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace FunctionPointInPolygonDetail
{
template <typename... Args>
using PointInPolygonCrossing = boost::geometry::strategy::within::crossings_multiply<Args...>;
template <typename... Args>
using PointInPolygonWinding = boost::geometry::strategy::within::winding<Args...>;
template <typename... Args>
using PointInPolygonFranklin = boost::geometry::strategy::within::franklin<Args...>;
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImplWithPool(const IColumn & x, const IColumn & 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;
template <template <typename...> typename Strategy>
auto factory = [& polygon]()
{
GeoUtils::normalizePolygon(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 = GeoUtils::serialize(polygon);
auto impl = known_polygons.get(serialized_polygon, factory);
return GeoUtils::pointInPolygon(x, y, *impl);
}
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImpl(const IColumn & x, const IColumn & y, Polygon & polygon)
{
PointInPolygonImpl impl(polygon);
return GeoUtils::pointInPolygon(x, y, impl);
}
}
template <template <typename> typename PointInPolygonImpl, bool use_object_pool = false>
class FunctionPointInPolygon : public IFunction
{
private:
using CoordinateType = Float64;
public:
template <typename Type>
using Point = boost::geometry::model::d2::point_xy<Type>;
template <typename Type>
using Polygon = boost::geometry::model::polygon<Point<Type>, false>;
template <typename Type>
using Box = boost::geometry::model::box<Point<Type>>;
static const char * name;
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionPointInPolygon<Strategy>>();
return std::make_shared<FunctionPointInPolygon<PointInPolygonImpl, use_object_pool>>();
}
String getName() const override
@ -56,132 +101,173 @@ public:
return 0;
}
void getReturnTypeAndPrerequisitesImpl(
const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, ExpressionActions::Actions & out_prerequisites) override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
{
throw Exception("Too few arguments", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION);
}
for (size_t i = 0; i < arguments.size(); ++i)
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(&*arguments[i].type);
auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr && i != 1)
throw Exception(getMsgPrefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto * tuple = checkAndGetDataType<DataTypeTuple>(array ? array->getNestedType().get() : arguments[i].get());
if (tuple == nullptr)
{
throw Exception("Argument " + toString(i + 1) + " for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
throw Exception(getMsgPrefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypes & elems = tuple->getElements();
const DataTypes & elements = tuple->getElements();
if (elems.size() != 2)
{
throw Exception("Tuple in argument " + toString(i + 1) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
}
if (elements.size() != 2)
throw Exception(getMsgPrefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
for (size_t elems_index = 0; elems_index < elems.size(); ++elems_index)
for (auto j : ext::range(0, elements.size()))
{
if (!checkDataType<DataTypeNumber<CoordinateType>>(&*elems[elems_index]))
if (!elements[j]->isNumeric())
{
throw Exception("Tuple element " + toString(elems_index + 1) + " in argument " + toString(i + 1)
+ " must be " + TypeName<CoordinateType>::get() + ".", ErrorCodes::BAD_ARGUMENTS);
throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
}
out_return_type = std::make_shared<DataTypeUInt8>();
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
using PointType = boost::geometry::model::d2::point_xy<CoordinateType>;
using PolygonType = boost::geometry::model::polygon<PointType>;
std::pair<CoordinateType, CoordinateType> min, max;
std::vector<PointType> polygon_points(arguments.size() - 1);
const IColumn * point_col = block.getByPosition(arguments[0]).column.get();
auto const_tuple_col = checkAndGetColumn<ColumnConst>(point_col);
if (const_tuple_col)
point_col = &const_tuple_col->getDataColumn();
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);
}
const auto & tuple_block = tuple_col->getData();
const auto & x = tuple_block.safeGetByPosition(0);
const auto & y = tuple_block.safeGetByPosition(1);
bool use_float64 = checkDataType<DataTypeFloat64>(x.type.get()) || checkDataType<DataTypeFloat64>(y.type.get());
auto & result_column = block.safeGetByPosition(result).column;
if (use_float64)
result_column = executeForType<Float64>(*x.column, *y.column, block, arguments);
else
result_column = executeForType<Float32>(*x.column, *y.column, block, arguments);
if (const_tuple_col)
result_column = std::make_shared<ColumnConst>(result_column, const_tuple_col->size());
}
private:
Float64 getCoordinateFromField(const Field & field)
{
switch (field.getType())
{
case Field::Types::Float64:
return field.get<Float64>();
case Field::Types::Int64:
return field.get<Int64>();
case Field::Types::UInt64:
return field.get<UInt64>();
default:
{
std::string msg = "Expected numeric field, but got ";
throw Exception(msg + Field::Types::toString(field.getType()), ErrorCodes::LOGICAL_ERROR);
}
}
}
template <typename Type>
ColumnPtr executeForType(const IColumn & x, const IColumn & y, Block & block, const ColumnNumbers & arguments)
{
Polygon<Type> polygon;
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
auto const_tuple_col = checkAndGetColumnConst<ColumnTuple>(block.getByPosition(arguments[i]).column.get());
if (!const_tuple_col)
auto const_col = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[i]).column.get());
auto array_col = const_col ? checkAndGetColumn<ColumnArray>(&const_col->getDataColumn()) : nullptr;
auto tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
if (!tuple_col)
throw Exception(getMsgPrefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN);
const auto & tuple_block = tuple_col->getData();
const auto & column_x = tuple_block.safeGetByPosition(0).column;
const auto & column_y = tuple_block.safeGetByPosition(1).column;
if (!polygon.outer().empty())
polygon.inners().emplace_back();
auto & container = polygon.outer().empty() ? polygon.outer() : polygon.inners().back();
auto size = column_x->size();
if (size == 0)
throw Exception(getMsgPrefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
for (auto j : ext::range(0, size))
{
throw Exception("Argument " + toString(i + 1) + " for function " + getName() + " must be constant tuple.", ErrorCodes::ILLEGAL_COLUMN);
Type x_coord = getCoordinateFromField((*column_x)[j]);
Type y_coord = getCoordinateFromField((*column_y)[j]);
container.push_back(Point<Type>(x_coord, y_coord));
}
TupleBackend data = const_tuple_col->getValue<Tuple>();
const CoordinateType x = data[0].get<Float64>();
const CoordinateType y = data[1].get<Float64>();
polygon_points[i - 1] = PointType(x, y);
if (i == 1)
{
min.first = x;
min.second = y;
max.first = x;
max.second = y;
}
else
{
min.first = std::min(min.first, x);
max.first = std::max(max.first, x);
min.second = std::min(min.second, y);
max.second = std::max(max.second, y);
}
/// Polygon assumed to be closed. Allow user to escape repeating of first point.
if (!boost::geometry::equals(container.front(), container.back()))
container.push_back(container.front());
}
PolygonType polygon;
boost::geometry::assign_points(polygon, polygon_points);
auto callImpl = use_object_pool
? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool<Polygon<Type>, PointInPolygonImpl<Type>>
: FunctionPointInPolygonDetail::callPointInPolygonImpl<Polygon<Type>, PointInPolygonImpl<Type>>;
Strategy<PointType> strategy;
auto point_checker = [&](CoordinateType x, CoordinateType y) -> bool
{
if (x < min.first || x > max.first || y < min.second || y > max.second)
return false;
PointType point(x, y);
return boost::geometry::covered_by(point, polygon, strategy);
};
size_t rows = block.rows();
auto point_column_const = checkAndGetColumnConst<ColumnTuple>(block.getByPosition(arguments[0]).column.get());
if (point_column_const)
{
TupleBackend data = point_column_const->getValue<Tuple>();
const CoordinateType point_x = data[0].get<Float64>();
const CoordinateType point_y = data[1].get<Float64>();
UInt8 value = point_checker(point_x, point_y);
block.getByPosition(result).column = DataTypeUInt8().createConstColumn(rows, UInt64(value));
return;
}
auto & res = block.getByPosition(result);
res.column = std::make_shared<ColumnUInt8>(rows);
IColumn & result_column = *res.column;
auto & result_data = static_cast<ColumnUInt8 &>(result_column).getData();
auto point_column = checkAndGetColumn<ColumnTuple>(block.getByPosition(arguments[0]).column.get());
auto column_x = checkAndGetColumn<ColumnVector<CoordinateType>>(point_column->getData().getByPosition(0).column.get());
auto column_y = checkAndGetColumn<ColumnVector<CoordinateType>>(point_column->getData().getByPosition(1).column.get());
for (size_t i = 0; i < rows; ++i)
{
const CoordinateType point_x = column_x->getElement(i);
const CoordinateType point_y = column_y->getElement(i);
result_data[i] = point_checker(point_x, point_y);
}
return callImpl(x, y, polygon);
}
};
template <typename Type>
using Point = boost::geometry::model::d2::point_xy<Type>;
template <typename Type>
using PointInPolygonCrossingStrategy = boost::geometry::strategy::within::crossings_multiply<Point<Type>>;
template <typename Type>
using PointInPolygonWindingStrategy = boost::geometry::strategy::within::winding<Point<Type>>;
template <typename Type>
using PointInPolygonFranklinStrategy = boost::geometry::strategy::within::franklin<Point<Type>>;
template <typename Type>
using PointInPolygonCrossing = GeoUtils::PointInPolygon<PointInPolygonCrossingStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonWinding = GeoUtils::PointInPolygon<PointInPolygonWindingStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonFranklin = GeoUtils::PointInPolygon<PointInPolygonFranklinStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonWithGrid = GeoUtils::PointInPolygonWithGrid<Type>;
template <>
const char * FunctionPointInPolygon<PointInPolygonCrossing>::name = "pointInPolygon";
template <>
const char * FunctionPointInPolygon<PointInPolygonWinding>::name = "pointInPolygonWinding";
template <>
const char * FunctionPointInPolygon<PointInPolygonFranklin>::name = "pointInPolygonFranklin";
template <>
const char * FunctionPointInPolygon<PointInPolygonWithGrid, true>::name = "pointInPolygonWithGrid";
void registerFunctionsGeo(FunctionFactory & factory)
@ -192,5 +278,6 @@ void registerFunctionsGeo(FunctionFactory & factory)
factory.registerFunction<FunctionPointInPolygon<PointInPolygonFranklin>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWinding>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonCrossing>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid, true>>();
}
}

View File

@ -0,0 +1,653 @@
#pragma once
#include <Core/Types.h>
#include <Core/Defines.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Core/TypeListNumber.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
/// Warning in boost::geometry during template strategy substitution.
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
#include <boost/geometry.hpp>
#if !__clang__
#pragma GCC diagnostic pop
#endif
#include <boost/geometry/geometries/point_xy.hpp>
#include <boost/geometry/geometries/polygon.hpp>
#include <boost/geometry/geometries/multi_polygon.hpp>
#include <boost/geometry/geometries/segment.hpp>
#include <boost/geometry/algorithms/comparable_distance.hpp>
#include <boost/geometry/strategies/cartesian/distance_pythagoras.hpp>
#include <array>
#include <vector>
#include <iterator>
#include <cmath>
#include <algorithm>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace GeoUtils
{
template <typename Polygon>
UInt64 getPolygonAllocatedSize(Polygon && polygon)
{
UInt64 size = 0;
using RingType = typename std::decay<Polygon>::type::ring_type;
auto addSizeOfRing = [& size](const RingType & ring) { size += sizeof(ring) + ring.capacity(); };
addSizeOfRing(polygon.outer());
const auto & inners = polygon.inners();
size += sizeof(inners) + inners.capacity();
for (auto & inner : inners)
addSizeOfRing(inner);
return size;
}
template <typename MultiPolygon>
UInt64 getMultiPolygonAllocatedSize(MultiPolygon && multi_polygon)
{
UInt64 size = multi_polygon.capacity();
for (const auto & polygon : multi_polygon)
size += getPolygonAllocatedSize(polygon);
return size;
}
template <typename CoordinateType = Float32, UInt16 gridHeight = 8, UInt16 gridWidth = 8>
class PointInPolygonWithGrid
{
public:
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
/// Counter-Clockwise ordering.
using Polygon = boost::geometry::model::polygon<Point, false>;
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon>;
using Box = boost::geometry::model::box<Point>;
using Segment = boost::geometry::model::segment<Point>;
explicit PointInPolygonWithGrid(const Polygon & polygon) : polygon(polygon) {}
inline void init();
bool hasEmptyBound() const { return has_empty_bound; }
UInt64 getAllocatedBytes() const;
inline bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y);
private:
enum class CellType
{
inner,
outer,
singleLine,
pairOfLinesSinglePolygon,
pairOfLinesDifferentPolygons,
complexPolygon
};
struct HalfPlane
{
/// Line
CoordinateType a;
CoordinateType b;
CoordinateType c;
void fill(const Point & from, const Point & to)
{
a = -(to.y() - from.y());
b = to.x() - from.x();
c = -from.x() * a - from.y() * b;
}
/// Inner part of the HalfPlane is the left side of initialized vector.
bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y) const { return a * x + b * y + c >= 0; }
};
struct Cell
{
static const int maxStoredHalfPlanes = 2;
HalfPlane half_planes[maxStoredHalfPlanes];
size_t index_of_inner_polygon;
CellType type;
};
Polygon polygon;
std::array<Cell, gridHeight * gridWidth> cells;
std::vector<MultiPolygon> polygons;
CoordinateType cell_width;
CoordinateType cell_height;
CoordinateType x_shift;
CoordinateType y_shift;
CoordinateType x_scale;
CoordinateType y_scale;
bool has_empty_bound = false;
bool was_grid_built = false;
void buildGrid();
void calcGridAttributes(Box & box);
template <typename T>
T ALWAYS_INLINE getCellIndex(T row, T col) const { return row * gridWidth + col; }
/// Complex case. Will check intersection directly.
inline void addComplexPolygonCell(size_t index, const Box & box);
/// Empty intersection or intersection == box.
inline void addCell(size_t index, const Box & empty_box);
/// Intersection is a single polygon.
inline void addCell(size_t index, const Box & box, const Polygon & intersection);
/// Intersection is a pair of polygons.
inline void addCell(size_t index, const Box & box, const Polygon & first, const Polygon & second);
/// Returns a list of half-planes were formed from intersection edges without box edges.
inline std::vector<HalfPlane> findHalfPlanes(const Box & box, const Polygon & intersection);
using Distance = typename boost::geometry::default_comparable_distance_result<Point, Segment>::type;
/// min(distance(point, edge) : edge in polygon)
inline Distance distance(const Point & point, const Polygon & polygon);
};
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
UInt64 PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::getAllocatedBytes() const
{
UInt64 size = sizeof(*this);
size += polygons.capacity();
for (const auto & polygon : polygons)
size += getMultiPolygonAllocatedSize(polygon);
size += getPolygonAllocatedSize(polygon);
return size;
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::init()
{
if (!was_grid_built)
buildGrid();
was_grid_built = true;
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::calcGridAttributes(
PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & box)
{
boost::geometry::envelope(polygon, box);
const Point & min_corner = box.min_corner();
const Point & max_corner = box.max_corner();
cell_width = (max_corner.x() - min_corner.x()) / gridWidth;
cell_height = (max_corner.y() - min_corner.y()) / gridHeight;
if (cell_width == 0 || cell_height == 0)
{
has_empty_bound = true;
return;
}
x_scale = 1 / cell_width;
y_scale = 1 / cell_height;
x_shift = -min_corner.x();
y_shift = -min_corner.y();
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::buildGrid()
{
Box box;
calcGridAttributes(box);
const Point & min_corner = box.min_corner();
for (size_t row = 0; row < gridHeight; ++row)
{
CoordinateType y_min = min_corner.y() + row * cell_height;
CoordinateType y_max = min_corner.y() + (row + 1) * cell_height;
for (size_t col = 0; col < gridWidth; ++col)
{
CoordinateType x_min = min_corner.x() + col * cell_width;
CoordinateType x_max = min_corner.x() + (col + 1) * cell_width;
Box cell_box(Point(x_min, y_min), Point(x_max, y_max));
Polygon cell_bound;
boost::geometry::convert(cell_box, cell_bound);
MultiPolygon intersection;
boost::geometry::intersection(polygon, cell_bound, intersection);
size_t cellIndex = getCellIndex(row, col);
if (intersection.empty())
addCell(cellIndex, cell_box);
else if (intersection.size() == 1)
addCell(cellIndex, cell_box, intersection.front());
else if (intersection.size() == 2)
addCell(cellIndex, cell_box, intersection.front(), intersection.back());
else
addComplexPolygonCell(cellIndex, cell_box);
}
}
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
bool PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::contains(CoordinateType x, CoordinateType y)
{
CoordinateType float_row = (y + y_shift) * y_scale;
CoordinateType float_col = (x + x_shift) * x_scale;
if (float_row < 0 || float_row > gridHeight)
return false;
if (float_col < 0 || float_col > gridWidth)
return false;
int row = std::min<int>(float_row, gridHeight - 1);
int col = std::min<int>(float_col, gridWidth - 1);
int index = getCellIndex(row, col);
const auto & cell = cells[index];
switch (cell.type)
{
case CellType::inner:
return true;
case CellType::outer:
return false;
case CellType::singleLine:
return cell.half_planes[0].contains(x, y);
case CellType::pairOfLinesSinglePolygon:
return cell.half_planes[0].contains(x, y) && cell.half_planes[1].contains(x, y);
case CellType::pairOfLinesDifferentPolygons:
return cell.half_planes[0].contains(x, y) || cell.half_planes[1].contains(x, y);
case CellType::complexPolygon:
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
default:
return false;
}
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
typename PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Distance
PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::distance(
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Point & point,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Polygon & polygon)
{
const auto & outer = polygon.outer();
Distance distance = 0;
for (auto i : ext::range(0, outer.size() - 1))
{
Segment segment(outer[i], outer[i + 1]);
Distance current = boost::geometry::comparable_distance(point, segment);
distance = i ? std::min(current, distance) : current;
}
return distance;
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
std::vector<typename PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::HalfPlane>
PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::findHalfPlanes(
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & box,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Polygon & intersection)
{
std::vector<HalfPlane> half_planes;
Polygon bound;
boost::geometry::convert(box, bound);
const auto & outer = intersection.outer();
for (auto i : ext::range(0, outer.size() - 1))
{
/// Want to detect is intersection edge was formed from box edge or from polygon edge.
/// If center of the edge closer to box, than don't form the half-plane.
Segment segment(outer[i], outer[i + 1]);
Point center((segment.first.x() + segment.second.x()) / 2, (segment.first.y() + segment.second.y()) / 2);
if (distance(center, polygon) < distance(center, bound))
{
half_planes.push_back({});
half_planes.back().fill(segment.first, segment.second);
}
}
return half_planes;
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::addComplexPolygonCell(
size_t index, const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & box)
{
cells[index].type = CellType::complexPolygon;
cells[index].index_of_inner_polygon = polygons.size();
/// Expand box in (1 + eps_factor) times to eliminate errors for points on box bound.
static constexpr float eps_factor = 0.01;
float x_eps = eps_factor * (box.max_corner().x() - box.min_corner().x());
float y_eps = eps_factor * (box.max_corner().y() - box.min_corner().y());
Point min_corner(box.min_corner().x() - x_eps, box.min_corner().y() - y_eps);
Point max_corner(box.max_corner().x() + x_eps, box.max_corner().y() + y_eps);
Box box_with_eps_bound(min_corner, max_corner);
Polygon bound;
boost::geometry::convert(box_with_eps_bound, bound);
MultiPolygon intersection;
boost::geometry::intersection(polygon, bound, intersection);
polygons.push_back(intersection);
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::addCell(
size_t index, const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & empty_box)
{
const auto & min_corner = empty_box.min_corner();
const auto & max_corner = empty_box.max_corner();
Point center((min_corner.x() + max_corner.x()) / 2, (min_corner.y() + max_corner.y()) / 2);
if (boost::geometry::within(center, polygon))
cells[index].type = CellType::inner;
else
cells[index].type = CellType::outer;
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::addCell(
size_t index,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & box,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Polygon & intersection)
{
if (!intersection.inners().empty())
addComplexPolygonCell(index, box);
auto half_planes = findHalfPlanes(box, intersection);
if (half_planes.empty())
addCell(index, box);
else if (half_planes.size() == 1)
{
cells[index].type = CellType::singleLine;
cells[index].half_planes[0] = half_planes[0];
}
else if (half_planes.size() == 2)
{
cells[index].type = CellType::pairOfLinesSinglePolygon;
cells[index].half_planes[0] = half_planes[0];
cells[index].half_planes[1] = half_planes[1];
}
else
addComplexPolygonCell(index, box);
}
template <typename CoordinateType, UInt16 gridHeight, UInt16 gridWidth>
void PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::addCell(
size_t index,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Box & box,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Polygon & first,
const PointInPolygonWithGrid<CoordinateType, gridHeight, gridWidth>::Polygon & second)
{
if (!first.inners().empty() || !second.inners().empty())
addComplexPolygonCell(index, box);
auto first_half_planes = findHalfPlanes(box, first);
auto second_half_planes = findHalfPlanes(box, second);
if (first_half_planes.empty())
addCell(index, box, first);
else if (second_half_planes.empty())
addCell(index, box, second);
else if (first_half_planes.size() == 1 && second_half_planes.size() == 1)
{
cells[index].type = CellType::pairOfLinesDifferentPolygons;
cells[index].half_planes[0] = first_half_planes[0];
cells[index].half_planes[1] = second_half_planes[0];
}
else
addComplexPolygonCell(index, box);
}
template <typename Strategy, typename CoordinateType = Float32>
class PointInPolygon
{
public:
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
/// Counter-Clockwise ordering.
using Polygon = boost::geometry::model::polygon<Point, false>;
using Box = boost::geometry::model::box<Point>;
explicit PointInPolygon(const Polygon & polygon) : polygon(polygon) {}
void init()
{
boost::geometry::envelope(polygon, box);
const Point & min_corner = box.min_corner();
const Point & max_corner = box.max_corner();
if (min_corner.x() == max_corner.x() || min_corner.y() == max_corner.y())
has_empty_bound = true;
}
bool hasEmptyBound() const { return has_empty_bound; }
inline bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y)
{
Point point(x, y);
if (!boost::geometry::within(point, box))
return false;
return boost::geometry::covered_by(point, polygon, strategy);
}
UInt64 getAllocatedBytes() const { return sizeof(*this); }
private:
const Polygon & polygon;
Box box;
bool has_empty_bound = false;
Strategy strategy;
};
/// Algorithms.
template <typename T, typename U, typename PointInPolygonImpl>
ColumnPtr pointInPolygon(const ColumnVector<T> & x, const ColumnVector<U> & y, PointInPolygonImpl && impl)
{
auto size = x.size();
impl.init();
if (impl.hasEmptyBound())
{
return std::make_shared<ColumnVector<UInt8>>(size, 0);
}
auto result = std::make_shared<ColumnVector<UInt8>>(size);
auto & data = result->getData();
const auto & x_data = x.getData();
const auto & y_data = y.getData();
for (auto i : ext::range(0, size))
{
data[i] = static_cast<UInt8>(impl.contains(x_data[i], y_data[i]));
}
return result;
}
template <typename ... Types>
struct CallPointInPolygon;
template <typename Type, typename ... Types>
struct CallPointInPolygon<Type, Types ...>
{
template <typename T, typename PointInPolygonImpl>
static ColumnPtr call(const ColumnVector<T> & x, const IColumn & y, PointInPolygonImpl && impl)
{
if (auto column = typeid_cast<const ColumnVector<Type> *>(&y))
return pointInPolygon(x, *column, impl);
return CallPointInPolygon<Types ...>::template call<T>(x, y, impl);
}
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNumbers>::Type;
if (auto column = typeid_cast<const ColumnVector<Type> *>(&x))
return Impl::template call<Type>(*column, y, impl);
return CallPointInPolygon<Types ...>::call(x, y, impl);
}
};
template <>
struct CallPointInPolygon<>
{
template <typename T, typename PointInPolygonImpl>
static ColumnPtr call(const ColumnVector<T> & x, const IColumn & y, PointInPolygonImpl && impl)
{
throw Exception(std::string("Unknown numeric column type: ") + typeid(y).name(), ErrorCodes::LOGICAL_ERROR);
}
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
throw Exception(std::string("Unknown numeric column type: ") + typeid(x).name(), ErrorCodes::LOGICAL_ERROR);
}
};
template <typename PointInPolygonImpl>
ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNumbers>::Type;
return Impl::call(x, y, impl);
}
/// Total angle (signed) between neighbor vectors in linestring. Zero if linestring.size() < 2.
template <typename Linestring>
float calcLinestringRotation(const Linestring & points)
{
using Point = typename std::decay<decltype(*points.begin())>::type;
float rotation = 0;
auto sqrLength = [](const Point & point) { return point.x() * point.x() + point.y() * point.y(); };
auto vecProduct = [](const Point & from, const Point & to) { return from.x() * to.y() - from.y() * to.x(); };
auto getVector = [](const Point & from, const Point & to) -> Point
{
return Point(to.x() - from.x(), to.y() - from.y());
};
for (auto it = points.begin(); std::next(it) != points.end(); ++it)
{
if (it != points.begin())
{
auto prev = std::prev(it);
auto next = std::next(it);
Point from = getVector(*prev, *it);
Point to = getVector(*it, *next);
float sqr_from_len = sqrLength(from);
float sqr_to_len = sqrLength(to);
float sqr_len_product = (sqr_from_len * sqr_to_len);
if (std::isfinite(sqr_len_product))
{
float vec_prod = vecProduct(from, to);
float sin_ang = vec_prod * std::fabs(vec_prod) / sqr_len_product;
sin_ang = std::max(-1.f, std::min(1.f, sin_ang));
rotation += std::asin(sin_ang);
}
}
}
return rotation;
}
/// Make inner linestring counter-clockwise and outers clockwise oriented.
template <typename Polygon>
void normalizePolygon(Polygon && polygon)
{
auto & outer = polygon.outer();
if (calcLinestringRotation(outer) < 0)
std::reverse(outer.begin(), outer.end());
auto & inners = polygon.inners();
for (auto & inner : inners)
if (calcLinestringRotation(inner) > 0)
std::reverse(inner.begin(), inner.end());
}
template <typename Polygon>
std::string serialize(Polygon && polygon)
{
std::string result;
{
WriteBufferFromString buffer(result);
using RingType = typename std::decay<Polygon>::type::ring_type;
auto serializeFloat = [&buffer](float value) { buffer.write(reinterpret_cast<char *>(&value), sizeof(value)); };
auto serializeSize = [&buffer](size_t size) { buffer.write(reinterpret_cast<char *>(&size), sizeof(size)); };
auto serializeRing = [& buffer, & serializeFloat, & serializeSize](const RingType & ring)
{
serializeSize(ring.size());
for (const auto & point : ring)
{
serializeFloat(point.x());
serializeFloat(point.y());
}
};
serializeRing(polygon.outer());
const auto & inners = polygon.inners();
serializeSize(inners.size());
for (auto & inner : inners)
serializeRing(inner);
}
return result;
}
} /// GeoUtils
} /// DB

View File

@ -7,3 +7,59 @@
1
0
0
inner
1
outer
0
single line
0
1
shifted grid
0
1
pair of lines, single polygon
1
0
0
1
0
0
pair of lines, different polygons
0
0
1
1
0
0
1
0
complex polygon
0
1
0
1
0
1
polygon with holes
0
1
0
1
0
1
0
1
0
1
0
1
polygons with reversed direction
0
1
0
1
0
1
0
eps for complex polygon in grid
1

View File

@ -1,9 +1,75 @@
SELECT pointInPolygonFranklin(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonFranklin(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonFranklin(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT 'inner';
SELECT pointInPolygonWithGrid((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2), (6, 0)]);
SELECT 'outer';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT 'single line';
SELECT pointInPolygonWithGrid((4.1, 0.1), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT pointInPolygonWithGrid((4.9, 0.9), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT 'shifted grid';
SELECT pointInPolygonWithGrid((0., 0.), [(6., 1.), (8., 4.), (5., 8.), (1., 2.), (6., 1.)]);
SELECT pointInPolygonWithGrid((6., 5.), [(6., 1.), (8., 4.), (5., 8.), (1., 2.), (6., 1.)]);
SELECT 'pair of lines, single polygon';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((0.9, 0.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((0.1, 0.9), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.2, 2.2), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.1, 2.9), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.9, 2.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT 'pair of lines, different polygons';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((1., 1.), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.7, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.1, 0.7), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((1.1, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.1, 1.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((5.0, 5.0), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((7.9, 7.9), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT 'complex polygon';
SELECT pointInPolygonWithGrid((0.05, 0.05), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.15, 0.15), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.3, 0.4), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.4, 0.7), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.7, 0.6), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.9, 0.1), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT 'polygon with holes';
SELECT pointInPolygonWithGrid((1., 1.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((2.5, 2.5), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((4., 4.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((4., 2.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((9., 9.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((0.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((1.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((2.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((3.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((4.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((5.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((6.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT 'polygons with reversed direction';
SELECT pointInPolygonWithGrid((4.1, .1), [(6., 0.), (0., 2.), (5., 8.), (8., 4.)]);
SELECT pointInPolygonWithGrid((4.1, .9), [(6., 0.), (0., 2.), (5., 8.), (8., 4.)]);
SELECT pointInPolygonWithGrid((1., 1.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((2.5, 2.5), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((4., 4.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((4., 2.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((9., 9.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)],[(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT 'eps for complex polygon in grid';
SELECT pointInPolygonWithGrid((0., 0.), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1., 0.), (-6., -7.), (-7., -6.), (0., 1.)])