ClickHouse/src/Functions/geometryConverters.h

365 lines
9.8 KiB
C++
Raw Normal View History

2020-06-07 13:42:09 +00:00
#pragma once
2020-06-07 12:33:49 +00:00
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Types.h>
#include <boost/variant.hpp>
#include <boost/geometry/geometries/geometries.hpp>
#include <boost/geometry.hpp>
#include <boost/geometry/geometries/point_xy.hpp>
2020-06-07 14:28:46 +00:00
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
2021-02-16 19:50:34 +00:00
#include <Common/NaNUtils.h>
2020-06-07 14:28:46 +00:00
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeCustomGeo.h>
2020-06-07 16:47:56 +00:00
#include <IO/WriteHelpers.h>
#include <Interpreters/castColumn.h>
2020-06-07 14:28:46 +00:00
2021-02-18 19:51:19 +00:00
#include <cmath>
#include <common/logger_useful.h>
2021-01-19 17:16:10 +00:00
namespace DB
{
2020-06-07 12:33:49 +00:00
2020-06-07 14:28:46 +00:00
namespace ErrorCodes
{
2020-06-14 17:04:10 +00:00
extern const int BAD_ARGUMENTS;
2020-06-07 14:28:46 +00:00
}
2020-06-07 12:33:49 +00:00
2021-01-19 14:52:53 +00:00
template <typename Point>
2021-02-18 19:51:19 +00:00
using Ring = boost::geometry::model::ring<Point>;
2021-01-19 14:52:53 +00:00
template <typename Point>
2021-02-18 19:51:19 +00:00
using Polygon = boost::geometry::model::polygon<Point>;
2021-01-19 14:52:53 +00:00
template <typename Point>
2021-02-18 19:51:19 +00:00
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon<Point>>;
2021-01-19 14:52:53 +00:00
template <typename Point>
using Geometry = boost::variant<Point, Ring<Point>, Polygon<Point>, MultiPolygon<Point>>;
2021-02-19 18:09:38 +00:00
template <typename Point>
using Figure = boost::variant<Ring<Point>, Polygon<Point>, MultiPolygon<Point>>;
2021-01-19 14:52:53 +00:00
2021-02-18 19:51:19 +00:00
using CartesianPoint = boost::geometry::model::d2::point_xy<Float64>;
2021-01-19 14:52:53 +00:00
using CartesianRing = Ring<CartesianPoint>;
using CartesianPolygon = Polygon<CartesianPoint>;
using CartesianMultiPolygon = MultiPolygon<CartesianPoint>;
using CartesianGeometry = Geometry<CartesianPoint>;
2021-01-18 23:51:34 +00:00
2021-02-26 12:25:28 +00:00
// using SphericalPoint = boost::geometry::model::point<Float64, 2, boost::geometry::cs::Spherical<boost::geometry::degree>>;
using SphericalPoint = boost::geometry::model::point<Float64, 2, boost::geometry::cs::spherical_equatorial<boost::geometry::degree>>;
using SphericalRing = Ring<SphericalPoint>;
using SphericalPolygon = Polygon<SphericalPoint>;
using SphericalMultiPolygon = MultiPolygon<SphericalPoint>;
using SphericalGeometry = Geometry<SphericalPoint>;
2021-01-18 23:51:34 +00:00
2021-02-19 18:09:38 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class RingFromColumnConverter;
2021-02-19 18:09:38 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class PolygonFromColumnConverter;
2021-02-19 18:09:38 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class MultiPolygonFromColumnConverter;
2021-02-19 18:09:38 +00:00
2021-01-18 23:51:34 +00:00
/**
2021-01-19 17:16:10 +00:00
* Class which takes some boost type and returns a pair of numbers.
2021-02-26 12:25:28 +00:00
* They are (x,y) in case of cartesian coordinated and (lon,lat) in case of Spherical.
2021-01-18 23:51:34 +00:00
*/
2021-02-19 18:09:38 +00:00
template <typename Point>
2021-02-20 13:59:37 +00:00
class PointFromColumnConverter
2020-06-07 13:42:09 +00:00
{
public:
2021-02-20 13:59:37 +00:00
explicit PointFromColumnConverter(ColumnPtr col_) : col(col_)
2020-06-07 14:28:46 +00:00
{
}
2021-02-20 17:44:18 +00:00
std::vector<Point> convert() const
{
return convertImpl(0, col->size());
}
2021-01-18 23:51:34 +00:00
2020-06-07 13:42:09 +00:00
private:
2021-02-20 13:59:37 +00:00
std::vector<Point> convertImpl(size_t shift, size_t count) const;
2021-02-20 13:59:37 +00:00
friend class RingFromColumnConverter<Point>;
ColumnPtr col{nullptr};
2020-06-07 13:42:09 +00:00
};
2021-02-18 19:51:19 +00:00
2021-01-19 14:52:53 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class RingFromColumnConverter
2020-06-07 14:28:46 +00:00
{
public:
2021-02-20 13:59:37 +00:00
explicit RingFromColumnConverter(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-20 13:59:37 +00:00
, point_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:28:46 +00:00
{
}
2021-02-20 13:59:37 +00:00
std::vector<Ring<Point>> convert() const;
2020-06-07 14:28:46 +00:00
private:
2021-02-20 13:59:37 +00:00
friend class PointFromColumnConverter<Point>;
2021-01-19 22:07:10 +00:00
/// To prevent use-after-free and increase column lifetime.
ColumnPtr col{nullptr};
2021-02-20 13:59:37 +00:00
const PointFromColumnConverter<Point> point_converter{};
2020-06-07 14:28:46 +00:00
};
2021-01-19 14:52:53 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class PolygonFromColumnConverter
2020-06-07 14:58:34 +00:00
{
public:
2021-02-20 13:59:37 +00:00
explicit PolygonFromColumnConverter(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-20 13:59:37 +00:00
, ring_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:58:34 +00:00
{
}
2021-02-20 13:59:37 +00:00
std::vector<Polygon<Point>> convert() const;
2021-02-19 18:09:38 +00:00
2020-06-07 14:58:34 +00:00
private:
2021-02-20 13:59:37 +00:00
friend class MultiPolygonFromColumnConverter<Point>;
2021-02-19 18:09:38 +00:00
2021-01-19 22:07:10 +00:00
/// To prevent use-after-free and increase column lifetime.
ColumnPtr col{nullptr};
2021-02-20 13:59:37 +00:00
const RingFromColumnConverter<Point> ring_converter{};
2020-06-07 14:58:34 +00:00
};
2021-01-19 14:52:53 +00:00
template<class Point>
2021-02-20 13:59:37 +00:00
class MultiPolygonFromColumnConverter
2020-06-07 14:58:34 +00:00
{
public:
2021-02-20 13:59:37 +00:00
explicit MultiPolygonFromColumnConverter(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-20 13:59:37 +00:00
, polygon_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:58:34 +00:00
{}
2021-02-20 13:59:37 +00:00
std::vector<MultiPolygon<Point>> convert() const;
2020-06-07 14:58:34 +00:00
private:
2021-01-19 22:07:10 +00:00
/// To prevent use-after-free and increase column lifetime.
ColumnPtr col{nullptr};
2021-02-20 13:59:37 +00:00
const PolygonFromColumnConverter<Point> polygon_converter{};
2020-06-07 14:58:34 +00:00
};
2021-01-18 23:51:34 +00:00
2021-02-20 13:59:37 +00:00
extern template class PointFromColumnConverter<CartesianPoint>;
2021-02-26 12:25:28 +00:00
extern template class PointFromColumnConverter<SphericalPoint>;
2021-02-20 13:59:37 +00:00
extern template class RingFromColumnConverter<CartesianPoint>;
2021-02-26 12:25:28 +00:00
extern template class RingFromColumnConverter<SphericalPoint>;
2021-02-20 13:59:37 +00:00
extern template class PolygonFromColumnConverter<CartesianPoint>;
2021-02-26 12:25:28 +00:00
extern template class PolygonFromColumnConverter<SphericalPoint>;
2021-02-20 13:59:37 +00:00
extern template class MultiPolygonFromColumnConverter<CartesianPoint>;
2021-02-26 12:25:28 +00:00
extern template class MultiPolygonFromColumnConverter<SphericalPoint>;
2021-02-15 19:22:13 +00:00
2020-06-07 16:04:35 +00:00
2021-02-26 12:25:28 +00:00
/// To serialize Spherical or Cartesian point (a pair of numbers in both cases).
2021-01-19 14:52:53 +00:00
template <typename Point>
class PointSerializer
2020-06-14 17:04:10 +00:00
{
2020-06-14 17:37:15 +00:00
public:
PointSerializer()
2021-01-19 14:52:53 +00:00
: first(ColumnFloat64::create())
, second(ColumnFloat64::create())
, first_container(first->getData())
, second_container(second->getData())
2020-06-14 17:04:10 +00:00
{}
explicit PointSerializer(size_t n)
2021-01-19 14:52:53 +00:00
: first(ColumnFloat64::create(n))
, second(ColumnFloat64::create(n))
, first_container(first->getData())
, second_container(second->getData())
2020-06-14 17:04:10 +00:00
{}
void add(const Point & point)
2020-06-14 17:04:10 +00:00
{
first_container.emplace_back(point.template get<0>());
second_container.emplace_back(point.template get<1>());
2020-06-14 17:04:10 +00:00
}
ColumnPtr finalize()
{
Columns columns(2);
2021-01-19 14:52:53 +00:00
columns[0] = std::move(first);
columns[1] = std::move(second);
2020-06-14 17:04:10 +00:00
return ColumnTuple::create(columns);
}
2020-06-14 17:37:15 +00:00
private:
2021-01-19 14:52:53 +00:00
ColumnFloat64::MutablePtr first;
ColumnFloat64::MutablePtr second;
ColumnFloat64::Container & first_container;
ColumnFloat64::Container & second_container;
2020-06-14 17:37:15 +00:00
};
2021-02-20 13:59:37 +00:00
/// Serialize Point, Ring as Ring
2021-01-19 14:52:53 +00:00
template <typename Point>
class RingSerializer
2020-06-14 17:37:15 +00:00
{
public:
RingSerializer()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
explicit RingSerializer(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
void add(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size += ring.size();
offsets->insertValue(size);
for (const auto & point : ring)
point_serializer.add(point);
2020-06-14 17:37:15 +00:00
}
ColumnPtr finalize()
{
2021-02-11 20:37:05 +00:00
return ColumnArray::create(point_serializer.finalize(), std::move(offsets));
2020-06-14 17:37:15 +00:00
}
private:
2020-06-21 15:18:09 +00:00
size_t size = 0;
PointSerializer<Point> point_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2021-02-20 13:59:37 +00:00
/// Serialize Point, Ring, Polygon as Polygon
2021-01-19 14:52:53 +00:00
template <typename Point>
class PolygonSerializer
2020-06-14 17:37:15 +00:00
{
public:
PolygonSerializer()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
explicit PolygonSerializer(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
void add(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
ring_serializer.add(ring);
2020-06-14 17:37:15 +00:00
}
void add(const Polygon<Point> & polygon)
2020-06-14 17:37:15 +00:00
{
size += 1 + polygon.inners().size();
offsets->insertValue(size);
ring_serializer.add(polygon.outer());
2020-06-14 17:37:15 +00:00
for (const auto & ring : polygon.inners())
ring_serializer.add(ring);
2020-06-14 17:37:15 +00:00
}
ColumnPtr finalize()
{
2021-02-11 20:37:05 +00:00
return ColumnArray::create(ring_serializer.finalize(), std::move(offsets));
2020-06-14 17:37:15 +00:00
}
private:
2020-06-21 15:18:09 +00:00
size_t size = 0;
RingSerializer<Point> ring_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2021-02-20 13:59:37 +00:00
/// Serialize Point, Ring, Polygon, MultiPolygon as MultiPolygon
2021-01-19 14:52:53 +00:00
template <typename Point>
class MultiPolygonSerializer
2020-06-14 17:37:15 +00:00
{
public:
MultiPolygonSerializer()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
explicit MultiPolygonSerializer(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
void add(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
polygon_serializer.add(ring);
2020-06-14 17:37:15 +00:00
}
void add(const Polygon<Point> & polygon)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
polygon_serializer.add(polygon);
2020-06-14 17:37:15 +00:00
}
void add(const MultiPolygon<Point> & multi_polygon)
2020-06-14 17:37:15 +00:00
{
size += multi_polygon.size();
2020-06-14 17:37:15 +00:00
offsets->insertValue(size);
for (const auto & polygon : multi_polygon)
{
polygon_serializer.add(polygon);
2020-06-14 17:37:15 +00:00
}
}
ColumnPtr finalize()
{
2021-02-11 20:37:05 +00:00
return ColumnArray::create(polygon_serializer.finalize(), std::move(offsets));
2020-06-14 17:37:15 +00:00
}
private:
2020-06-21 15:18:09 +00:00
size_t size = 0;
PolygonSerializer<Point> polygon_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2020-06-14 17:04:10 +00:00
2021-02-19 22:48:59 +00:00
template <typename PType>
2021-02-20 13:59:37 +00:00
struct ConverterType
2021-02-19 22:48:59 +00:00
{
using Type = PType;
};
template <typename Point, typename F>
static void callOnGeometryDataType(DataTypePtr type, F && f)
2020-06-14 17:04:10 +00:00
{
2021-02-20 13:59:37 +00:00
/// There is no Point type, because for most of geometry functions it is useless.
2021-02-20 17:44:18 +00:00
if (DataTypeCustomPointSerialization::nestedDataType()->equals(*type))
return f(ConverterType<PointFromColumnConverter<Point>>());
else if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
2021-02-20 13:59:37 +00:00
return f(ConverterType<RingFromColumnConverter<Point>>());
2021-02-20 17:44:18 +00:00
else if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
2021-02-20 13:59:37 +00:00
return f(ConverterType<PolygonFromColumnConverter<Point>>());
2021-02-20 17:44:18 +00:00
else if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
2021-02-20 13:59:37 +00:00
return f(ConverterType<MultiPolygonFromColumnConverter<Point>>());
throw Exception(fmt::format("Unknown geometry type {}", type->getName()), ErrorCodes::BAD_ARGUMENTS);
}
2021-01-19 14:52:53 +00:00
2020-06-14 17:04:10 +00:00
template <typename Point, typename F>
static void callOnTwoGeometryDataTypes(DataTypePtr left_type, DataTypePtr right_type, F && func)
{
return callOnGeometryDataType<Point>(left_type, [&](const auto & left_types)
{
2021-02-20 13:59:37 +00:00
using LeftConverterType = std::decay_t<decltype(left_types)>;
2021-02-15 19:22:13 +00:00
return callOnGeometryDataType<Point>(right_type, [&](const auto & right_types)
{
2021-02-20 13:59:37 +00:00
using RightConverterType = std::decay_t<decltype(right_types)>;
2021-02-15 19:22:13 +00:00
2021-02-20 13:59:37 +00:00
return func(LeftConverterType(), RightConverterType());
});
});
}
2021-02-19 18:09:38 +00:00
2020-06-07 12:33:49 +00:00
}