ClickHouse

Форк
0
/
pointInPolygon.cpp 
581 строка · 22.6 Кб
1
#include <Functions/FunctionFactory.h>
2
#include <Functions/PolygonUtils.h>
3
#include <Functions/FunctionHelpers.h>
4

5
#include <boost/geometry.hpp>
6
#include <boost/geometry/geometries/point_xy.hpp>
7
#include <boost/geometry/geometries/polygon.hpp>
8

9
#include <Columns/ColumnArray.h>
10
#include <Columns/ColumnFixedString.h>
11
#include <Columns/ColumnString.h>
12
#include <Columns/ColumnTuple.h>
13
#include <Columns/ColumnsNumber.h>
14
#include <Common/ObjectPool.h>
15
#include <Common/ProfileEvents.h>
16
#include <base/arithmeticOverflow.h>
17
#include <DataTypes/DataTypeArray.h>
18
#include <DataTypes/DataTypeString.h>
19
#include <DataTypes/DataTypeTuple.h>
20
#include <DataTypes/DataTypesNumber.h>
21
#include <IO/WriteHelpers.h>
22
#include <Interpreters/ExpressionActions.h>
23
#include <Interpreters/Context.h>
24
#include <Interpreters/castColumn.h>
25

26
#include <string>
27
#include <memory>
28

29

30
namespace ProfileEvents
31
{
32
    extern const Event PolygonsAddedToPool;
33
    extern const Event PolygonsInPoolAllocatedBytes;
34
}
35

36
namespace DB
37
{
38
namespace ErrorCodes
39
{
40
    extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
41
    extern const int BAD_ARGUMENTS;
42
    extern const int ILLEGAL_TYPE_OF_ARGUMENT;
43
    extern const int ILLEGAL_COLUMN;
44
}
45

46
namespace
47
{
48

49
using CoordinateType = Float64;
50
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
51
using Polygon = boost::geometry::model::polygon<Point, false>;
52
using Box = boost::geometry::model::box<Point>;
53

54

55
template <typename PointInConstPolygonImpl>
56
class FunctionPointInPolygon : public IFunction
57
{
58
public:
59
    static inline const char * name = "pointInPolygon";
60

61
    explicit FunctionPointInPolygon(bool validate_) : validate(validate_) {}
62

63
    static FunctionPtr create(ContextPtr context)
64
    {
65
        return std::make_shared<FunctionPointInPolygon<PointInConstPolygonImpl>>(
66
            context->getSettingsRef().validate_polygons);
67
    }
68

69
    String getName() const override
70
    {
71
        return name;
72
    }
73

74
    bool isVariadic() const override
75
    {
76
        return true;
77
    }
78

79
    size_t getNumberOfArguments() const override
80
    {
81
        return 0;
82
    }
83

84
    bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
85

86
    DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
87
    {
88
        if (arguments.size() < 2)
89
        {
90
            throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName());
91
        }
92

93
        /** We allow function invocation in one of the following forms:
94
          *
95
          * pointInPolygon((x, y), [(x1, y1), (x2, y2), ...])
96
          * - simple polygon
97
          * pointInPolygon((x, y), [(x1, y1), (x2, y2), ...], [(x21, y21), (x22, y22), ...], ...)
98
          * - polygon with a number of holes, each hole as a subsequent argument.
99
          * pointInPolygon((x, y), [[(x1, y1), (x2, y2), ...], [(x21, y21), (x22, y22), ...], ...])
100
          * - polygon with a number of holes, all as multidimensional array
101
          */
102

103
        auto validate_tuple = [this](size_t i, const DataTypeTuple * tuple)
104
        {
105
            if (tuple == nullptr)
106
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} must contain a tuple", getMessagePrefix(i));
107

108
            const DataTypes & elements = tuple->getElements();
109

110
            if (elements.size() != 2)
111
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} must have exactly two elements", getMessagePrefix(i));
112

113
            for (auto j : collections::range(0, elements.size()))
114
            {
115
                if (!isNativeNumber(elements[j]))
116
                {
117
                    throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} must contain numeric tuple at position {}",
118
                                    getMessagePrefix(i), j + 1);
119
                }
120
            }
121
        };
122

123
        validate_tuple(0, checkAndGetDataType<DataTypeTuple>(arguments[0].get()));
124

125
        if (arguments.size() == 2)
126
        {
127
            const auto * array = checkAndGetDataType<DataTypeArray>(arguments[1].get());
128
            if (array == nullptr)
129
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} must contain an array of tuples or an array of arrays of tuples.", getMessagePrefix(1));
130

131
            const auto * nested_array = checkAndGetDataType<DataTypeArray>(array->getNestedType().get());
132
            if (nested_array != nullptr)
133
            {
134
                array = nested_array;
135
            }
136

137
            validate_tuple(1, checkAndGetDataType<DataTypeTuple>(array->getNestedType().get()));
138
        }
139
        else
140
        {
141
            for (size_t i = 1; i < arguments.size(); ++i)
142
            {
143
                const auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
144
                if (array == nullptr)
145
                    throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} must contain an array of tuples", getMessagePrefix(i));
146

147
                validate_tuple(i, checkAndGetDataType<DataTypeTuple>(array->getNestedType().get()));
148
            }
149
        }
150

151
        return std::make_shared<DataTypeUInt8>();
152
    }
153

154
    ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
155
    {
156
        const IColumn * point_col = arguments[0].column.get();
157
        const auto * const_tuple_col = checkAndGetColumn<ColumnConst>(point_col);
158
        if (const_tuple_col)
159
            point_col = &const_tuple_col->getDataColumn();
160

161
        const auto * tuple_col = checkAndGetColumn<ColumnTuple>(point_col);
162
        if (!tuple_col)
163
            throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument for function {} must be constant array of tuples.",
164
                            getName());
165

166
        const auto & tuple_columns = tuple_col->getColumns();
167

168
        const ColumnWithTypeAndName & poly = arguments[1];
169
        const IColumn * poly_col = poly.column.get();
170
        const ColumnConst * const_poly_col = checkAndGetColumn<ColumnConst>(poly_col);
171

172
        bool point_is_const = const_tuple_col != nullptr;
173
        bool poly_is_const = const_poly_col != nullptr;
174

175
        /// Two different algorithms are used for constant and non constant polygons.
176
        /// Constant polygons are preprocessed to speed up matching.
177
        /// For non-constant polygons, we cannot spend time for preprocessing
178
        ///  and have to quickly match on the fly without creating temporary data structures.
179

180
        if (poly_is_const)
181
        {
182
            Polygon polygon;
183
            parseConstPolygon(arguments, polygon);
184

185
            /// Polygons are preprocessed and saved in cache.
186
            /// Preprocessing can be computationally heavy but dramatically speeds up matching.
187

188
            using Pool = ObjectPoolMap<PointInConstPolygonImpl, UInt128>;
189
            /// C++11 has thread-safe function-local static.
190
            static Pool known_polygons;
191

192
            auto factory = [&polygon]()
193
            {
194
                auto ptr = std::make_unique<PointInConstPolygonImpl>(polygon);
195

196
                ProfileEvents::increment(ProfileEvents::PolygonsAddedToPool);
197
                ProfileEvents::increment(ProfileEvents::PolygonsInPoolAllocatedBytes, ptr->getAllocatedBytes());
198

199
                return ptr.release();
200
            };
201

202
            auto impl = known_polygons.get(sipHash128(polygon), factory);
203

204
            if (point_is_const)
205
            {
206
                bool is_in = impl->contains(tuple_columns[0]->getFloat64(0), tuple_columns[1]->getFloat64(0));
207
                return result_type->createColumnConst(input_rows_count, is_in);
208
            }
209
            else
210
            {
211
                return pointInPolygon(*tuple_columns[0], *tuple_columns[1], *impl);
212
            }
213
        }
214
        else
215
        {
216
            if (arguments.size() != 2)
217
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multi-argument version of function {} works only with const polygon",
218
                    getName());
219

220
            auto res_column = ColumnVector<UInt8>::create(input_rows_count);
221
            auto & data = res_column->getData();
222

223
            /// A polygon, possibly with holes, is represented by 2d array:
224
            /// [[(outer_x_1, outer_y_1, ...)], [(hole1_x_1, hole1_y_1), ...], ...]
225
            ///
226
            /// Or, a polygon without holes can be represented by 1d array:
227
            /// [(outer_x_1, outer_y_1, ...)]
228

229
            if (isTwoDimensionalArray(*arguments[1].type))
230
            {
231
                /// We cast everything to Float64 in advance (in batch fashion)
232
                ///  to avoid casting with virtual calls in a loop.
233
                /// Note that if the type is already Float64, the operation in noop.
234

235
                ColumnPtr polygon_column_float64 = castColumn(
236
                    arguments[1],
237
                    std::make_shared<DataTypeArray>(
238
                        std::make_shared<DataTypeArray>(
239
                            std::make_shared<DataTypeTuple>(DataTypes{
240
                                std::make_shared<DataTypeFloat64>(),
241
                                std::make_shared<DataTypeFloat64>()}))));
242

243
                for (size_t i = 0; i < input_rows_count; ++i)
244
                {
245
                    size_t point_index = point_is_const ? 0 : i;
246
                    data[i] = isInsidePolygonWithHoles(
247
                        tuple_columns[0]->getFloat64(point_index),
248
                        tuple_columns[1]->getFloat64(point_index),
249
                        *polygon_column_float64,
250
                        i);
251
                }
252
            }
253
            else
254
            {
255
                ColumnPtr polygon_column_float64 = castColumn(
256
                    arguments[1],
257
                    std::make_shared<DataTypeArray>(
258
                        std::make_shared<DataTypeTuple>(DataTypes{
259
                            std::make_shared<DataTypeFloat64>(),
260
                            std::make_shared<DataTypeFloat64>()})));
261

262
                for (size_t i = 0; i < input_rows_count; ++i)
263
                {
264
                    size_t point_index = point_is_const ? 0 : i;
265
                    data[i] = isInsidePolygonWithoutHoles(
266
                        tuple_columns[0]->getFloat64(point_index),
267
                        tuple_columns[1]->getFloat64(point_index),
268
                        *polygon_column_float64,
269
                        i);
270
                }
271
            }
272

273
            return res_column;
274
        }
275
    }
276

277
private:
278
    bool validate;
279

280
    std::string getMessagePrefix(size_t i) const
281
    {
282
        return "Argument " + toString(i + 1) + " for function " + getName();
283
    }
284

285
    bool isTwoDimensionalArray(const IDataType & type) const
286
    {
287
        return WhichDataType(type).isArray()
288
            && WhichDataType(static_cast<const DataTypeArray &>(type).getNestedType()).isArray();
289
    }
290

291
    /// Implementation methods to check point-in-polygon on the fly (for non-const polygons).
292

293
    bool isInsideRing(
294
        Float64 point_x,
295
        Float64 point_y,
296
        const Float64 * ring_x_data,
297
        const Float64 * ring_y_data,
298
        size_t ring_begin,
299
        size_t ring_end) const
300
    {
301
        size_t size = ring_end - ring_begin;
302

303
        if (size < 2)
304
            return false;
305

306
        /** This is the algorithm by W. Randolph Franklin
307
          * https://wrf.ecse.rpi.edu//Research/Short_Notes/pnpoly.html
308
          *
309
          * Basically it works like this:
310
          * From the point, cast a horizontal ray to the right
311
          *  and count the number of intersections with polygon edges
312
          *  (every edge is considered semi-closed, e.g. includes the first vertex and does not include the last)
313
          *
314
          * Advantages:
315
          * - works regardless to the orientation;
316
          * - for polygon without holes:
317
          *   works regardless to whether the polygon is closed by last vertex equals to first vertex or not;
318
          *   (no need to preprocess polygon in any way)
319
          * - easy to apply for polygons with holes and for multi-polygons;
320
          * - it even works for polygons with self-intersections in a reasonable way;
321
          * - simplicity and performance;
322
          * - can be additionally speed up with loop unrolling and/or binary search for possible intersecting edges.
323
          *
324
          * Drawbacks:
325
          * - it's unspecified whether a point of the edge is inside or outside of a polygon
326
          *   (looks like it's inside for "left" edges and outside for "right" edges)
327
          *
328
          * Why not to apply the same algorithm available in boost::geometry?
329
          * It will require to move data from columns to temporary containers.
330
          * Despite the fact that the boost library is template based and allows arbitrary containers and points,
331
          *  it's diffucult to use without data movement because
332
          *  we use structure-of-arrays for coordinates instead of arrays-of-structures.
333
          */
334

335
        size_t vertex1_idx = ring_begin;
336
        size_t vertex2_idx = ring_end - 1;
337
        bool res = false;
338

339
        while (vertex1_idx < ring_end)
340
        {
341
            /// First condition checks that the point is inside horizontal row between edge top and bottom y-coordinate.
342
            /// Second condition checks for intersection with the edge.
343

344
            if (((ring_y_data[vertex1_idx] > point_y) != (ring_y_data[vertex2_idx] > point_y))
345
                && (point_x < (ring_x_data[vertex2_idx] - ring_x_data[vertex1_idx])
346
                    * (point_y - ring_y_data[vertex1_idx]) / (ring_y_data[vertex2_idx] - ring_y_data[vertex1_idx])
347
                    + ring_x_data[vertex1_idx]))
348
            {
349
                res = !res;
350
            }
351

352
            vertex2_idx = vertex1_idx;
353
            ++vertex1_idx;
354
        }
355

356
        return res;
357
    }
358

359
    bool isInsidePolygonWithoutHoles(
360
        Float64 point_x,
361
        Float64 point_y,
362
        const IColumn & polygon_column,
363
        size_t i) const
364
    {
365
        const auto & array_col = static_cast<const ColumnArray &>(polygon_column);
366

367
        size_t begin = array_col.getOffsets()[i - 1];
368
        size_t end = array_col.getOffsets()[i];
369
        size_t size = end - begin;
370

371
        if (size < 2)
372
            return false;
373

374
        const auto & tuple_columns = static_cast<const ColumnTuple &>(array_col.getData()).getColumns();
375
        const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
376
        const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
377

378
        return isInsideRing(point_x, point_y, x_data, y_data, begin, end);
379
    }
380

381
    bool isInsidePolygonWithHoles(
382
        Float64 point_x,
383
        Float64 point_y,
384
        const IColumn & polygon_column,
385
        size_t i) const
386
    {
387
        const auto & array_col = static_cast<const ColumnArray &>(polygon_column);
388
        size_t rings_begin = array_col.getOffsets()[i - 1];
389
        size_t rings_end = array_col.getOffsets()[i];
390

391
        const auto & nested_array_col = static_cast<const ColumnArray &>(array_col.getData());
392
        const auto & tuple_columns = static_cast<const ColumnTuple &>(nested_array_col.getData()).getColumns();
393
        const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
394
        const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
395

396
        for (size_t j = rings_begin; j < rings_end; ++j)
397
        {
398
            size_t begin = nested_array_col.getOffsets()[j - 1];
399
            size_t end = nested_array_col.getOffsets()[j];
400

401
            if (j == rings_begin)
402
            {
403
                if (!isInsideRing(point_x, point_y, x_data, y_data, begin, end))
404
                    return false;
405
            }
406
            else
407
            {
408
                if (isInsideRing(point_x, point_y, x_data, y_data, begin, end))
409
                    return false;
410
            }
411
        }
412

413
        return true;
414
    }
415

416
    /// Implementation methods to create boost::geometry::polygon for subsequent preprocessing.
417
    /// They are used to optimize matching for constant polygons. Preprocessing may take significant amount of time.
418

419
    template <typename T>
420
    void parseRing(
421
        const Float64 * x_data,
422
        const Float64 * y_data,
423
        size_t begin,
424
        size_t end,
425
        T & out_container) const
426
    {
427
        out_container.reserve(end - begin);
428
        for (size_t i = begin; i < end; ++i)
429
        {
430
            Int64 result = 0;
431
            if (common::mulOverflow(static_cast<Int64>(x_data[i]), static_cast<Int64>(y_data[i]), result))
432
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "The coordinates of the point are such that "
433
                                "subsequent calculations cannot be performed correctly. "
434
                                "Most likely they are very large in modulus.");
435

436
            out_container.emplace_back(x_data[i], y_data[i]);
437
        }
438
    }
439

440
    void parseConstPolygonWithoutHolesFromSingleColumn(const IColumn & column, size_t i, Polygon & out_polygon) const
441
    {
442
        const auto & array_col = static_cast<const ColumnArray &>(column);
443
        size_t begin = array_col.getOffsets()[i - 1];
444
        size_t end = array_col.getOffsets()[i];
445

446
        const auto & tuple_columns = static_cast<const ColumnTuple &>(array_col.getData()).getColumns();
447
        const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
448
        const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
449

450
        parseRing(x_data, y_data, begin, end, out_polygon.outer());
451
    }
452

453
    void parseConstPolygonWithHolesFromSingleColumn(const IColumn & column, size_t i, Polygon & out_polygon) const
454
    {
455
        const auto & array_col = static_cast<const ColumnArray &>(column);
456
        size_t rings_begin = array_col.getOffsets()[i - 1];
457
        size_t rings_end = array_col.getOffsets()[i];
458

459
        const auto & nested_array_col = static_cast<const ColumnArray &>(array_col.getData());
460
        const auto & tuple_columns = static_cast<const ColumnTuple &>(nested_array_col.getData()).getColumns();
461
        const auto * x_data = static_cast<const ColumnFloat64 &>(*tuple_columns[0]).getData().data();
462
        const auto * y_data = static_cast<const ColumnFloat64 &>(*tuple_columns[1]).getData().data();
463

464
        for (size_t j = rings_begin; j < rings_end; ++j)
465
        {
466
            size_t begin = nested_array_col.getOffsets()[j - 1];
467
            size_t end = nested_array_col.getOffsets()[j];
468

469
            if (out_polygon.outer().empty())
470
            {
471
                parseRing(x_data, y_data, begin, end, out_polygon.outer());
472
            }
473
            else
474
            {
475
                out_polygon.inners().emplace_back();
476
                parseRing(x_data, y_data, begin, end, out_polygon.inners().back());
477
            }
478
        }
479
    }
480

481
    void parseConstPolygonWithHolesFromMultipleColumns(const ColumnsWithTypeAndName & arguments, Polygon & out_polygon) const
482
    {
483
        for (size_t i = 1; i < arguments.size(); ++i)
484
        {
485
            const auto * const_col = checkAndGetColumn<ColumnConst>(arguments[i].column.get());
486
            if (!const_col)
487
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multi-argument version of function {} works only with const polygon",
488
                    getName());
489

490
            const auto * array_col = checkAndGetColumn<ColumnArray>(&const_col->getDataColumn());
491
            const auto * tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
492

493
            if (!tuple_col)
494
                throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{} must be constant array of tuples", getMessagePrefix(i));
495

496
            const auto & tuple_columns = tuple_col->getColumns();
497
            const auto & column_x = tuple_columns[0];
498
            const auto & column_y = tuple_columns[1];
499

500
            if (!out_polygon.outer().empty())
501
                out_polygon.inners().emplace_back();
502

503
            auto & container = out_polygon.outer().empty() ? out_polygon.outer() : out_polygon.inners().back();
504

505
            auto size = column_x->size();
506

507
            if (size == 0)
508
                throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{} shouldn't be empty.", getMessagePrefix(i));
509

510
            for (auto j : collections::range(0, size))
511
            {
512
                CoordinateType x_coord = column_x->getFloat64(j);
513
                CoordinateType y_coord = column_y->getFloat64(j);
514
                container.push_back(Point(x_coord, y_coord));
515
            }
516
        }
517
    }
518

519
    void parseConstPolygonFromSingleColumn(const ColumnsWithTypeAndName & arguments, Polygon & out_polygon) const
520
    {
521
        if (isTwoDimensionalArray(*arguments[1].type))
522
        {
523
            ColumnPtr polygon_column_float64 = castColumn(
524
                arguments[1],
525
                std::make_shared<DataTypeArray>(
526
                    std::make_shared<DataTypeArray>(
527
                        std::make_shared<DataTypeTuple>(DataTypes{
528
                            std::make_shared<DataTypeFloat64>(),
529
                            std::make_shared<DataTypeFloat64>()}))));
530

531
            const ColumnConst & column_const = typeid_cast<const ColumnConst &>(*polygon_column_float64);
532
            const IColumn & column_const_data = column_const.getDataColumn();
533

534
            parseConstPolygonWithHolesFromSingleColumn(column_const_data, 0, out_polygon);
535
        }
536
        else
537
        {
538
            ColumnPtr polygon_column_float64 = castColumn(
539
                arguments[1],
540
                std::make_shared<DataTypeArray>(
541
                    std::make_shared<DataTypeTuple>(DataTypes{
542
                        std::make_shared<DataTypeFloat64>(),
543
                        std::make_shared<DataTypeFloat64>()})));
544

545
            const ColumnConst & column_const = typeid_cast<const ColumnConst &>(*polygon_column_float64);
546
            const IColumn & column_const_data = column_const.getDataColumn();
547

548
            parseConstPolygonWithoutHolesFromSingleColumn(column_const_data, 0, out_polygon);
549
        }
550
    }
551

552
    void NO_SANITIZE_UNDEFINED parseConstPolygon(const ColumnsWithTypeAndName & arguments, Polygon & out_polygon) const
553
    {
554
        if (arguments.size() == 2)
555
            parseConstPolygonFromSingleColumn(arguments, out_polygon);
556
        else
557
            parseConstPolygonWithHolesFromMultipleColumns(arguments, out_polygon);
558

559
        /// Fix orientation and close rings. It's required for subsequent processing.
560
        boost::geometry::correct(out_polygon);
561

562
#if !defined(__clang_analyzer__) /// It does not like boost.
563
        if (validate)
564
        {
565
            std::string failure_message;
566
            auto is_valid = boost::geometry::is_valid(out_polygon, failure_message);
567
            if (!is_valid)
568
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "Polygon is not valid: {}", failure_message);
569
        }
570
#endif
571
    }
572
};
573

574
}
575

576
REGISTER_FUNCTION(PointInPolygon)
577
{
578
    factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid<Float64>>>();
579
}
580

581
}
582

Использование cookies

Мы используем файлы cookie в соответствии с Политикой конфиденциальности и Политикой использования cookies.

Нажимая кнопку «Принимаю», Вы даете АО «СберТех» согласие на обработку Ваших персональных данных в целях совершенствования нашего веб-сайта и Сервиса GitVerse, а также повышения удобства их использования.

Запретить использование cookies Вы можете самостоятельно в настройках Вашего браузера.