ClickHouse/src/Functions/geometryConverters.h

506 lines
14 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>
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
// TODO: maybe use isInfinite from clickhouse codebase
#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;
2021-02-16 19:50:34 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
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-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-18 19:51:19 +00:00
using GeographicPoint = boost::geometry::model::point<Float64, 2, boost::geometry::cs::geographic<boost::geometry::degree>>;
2021-01-19 14:52:53 +00:00
using GeographicRing = Ring<GeographicPoint>;
using GeographicPolygon = Polygon<GeographicPoint>;
using GeographicMultiPolygon = MultiPolygon<GeographicPoint>;
using GeographicGeometry = Geometry<GeographicPoint>;
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.
* They are (x,y) in case of cartesian coordinated and (lon,lat) in case of geographic.
2021-01-18 23:51:34 +00:00
*/
template <typename PointType>
class PointFromColumnParser
2020-06-07 13:42:09 +00:00
{
public:
2021-02-18 19:51:19 +00:00
using Container = std::conditional_t<std::is_same_v<PointType, CartesianPoint>, CartesianGeometry, GeographicGeometry>;
2021-02-11 20:37:05 +00:00
explicit PointFromColumnParser(ColumnPtr col_) : col(col_)
2020-06-07 14:28:46 +00:00
{
}
2021-02-18 19:51:19 +00:00
std::vector<PointType> parse(size_t shift, size_t count) const
2021-01-18 23:51:34 +00:00
{
2021-02-18 19:51:19 +00:00
const auto * tuple = typeid_cast<const ColumnTuple *>(col.get());
const auto & tuple_columns = tuple->getColumns();
2021-01-18 23:51:34 +00:00
2021-02-18 19:51:19 +00:00
const auto * x_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[0].get());
const auto * y_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[1].get());
2020-06-14 17:04:10 +00:00
2021-02-18 19:51:19 +00:00
const auto * first_container = x_data->getData().data() + shift;
const auto * second_container = y_data->getData().data() + shift;
2020-06-07 14:28:46 +00:00
2021-02-18 19:51:19 +00:00
std::vector<PointType> answer(count);
2020-06-07 16:04:35 +00:00
2021-02-18 19:51:19 +00:00
for (size_t i = 0; i < count; ++i)
{
const Float64 first = first_container[i];
const Float64 second = second_container[i];
if (isNaN(first) || isNaN(second))
throw Exception("Point's component must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (isinf(first) || isinf(second))
throw Exception("Point's component must not be infinite", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
answer[i] = PointType(first, second);
}
return answer;
2020-06-07 14:28:46 +00:00
}
2021-01-18 23:51:34 +00:00
2020-06-07 13:42:09 +00:00
private:
2021-01-19 22:07:10 +00:00
/// To prevent use-after-free and increase column lifetime.
2020-06-14 17:04:10 +00:00
ColumnPtr col;
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>
2020-06-07 14:28:46 +00:00
class RingFromColumnParser
{
public:
2021-02-11 20:37:05 +00:00
explicit RingFromColumnParser(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-18 19:51:19 +00:00
, offsets(typeid_cast<const ColumnArray &>(*col_).getOffsets())
, point_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:28:46 +00:00
{
}
2021-02-18 19:51:19 +00:00
std::vector<Ring<Point>> parse(size_t shift, size_t /*size*/) const
2020-06-07 16:04:35 +00:00
{
2021-02-18 19:51:19 +00:00
size_t prev_offset = shift;
2021-01-20 11:22:43 +00:00
2021-02-18 19:51:19 +00:00
std::vector<Ring<Point>> answer;
answer.reserve(offsets.size());
2020-06-07 14:28:46 +00:00
2021-02-18 19:51:19 +00:00
for (size_t offset : offsets)
2020-06-07 14:58:34 +00:00
{
2021-02-18 19:51:19 +00:00
offset += shift;
auto points = point_parser.parse(prev_offset, offset - prev_offset);
answer.emplace_back(points.begin(), points.end());
prev_offset = offset;
2020-06-07 14:28:46 +00:00
}
2021-02-18 19:51:19 +00:00
return answer;
2020-06-07 14:28:46 +00:00
}
private:
2021-01-19 22:07:10 +00:00
/// To prevent use-after-free and increase column lifetime.
ColumnPtr col;
2020-06-07 14:28:46 +00:00
const IColumn::Offsets & offsets;
2021-01-19 14:52:53 +00:00
const PointFromColumnParser<Point> point_parser;
2020-06-07 14:28:46 +00:00
};
2021-01-19 14:52:53 +00:00
template<class Point>
2020-06-07 14:58:34 +00:00
class PolygonFromColumnParser
{
public:
2021-02-11 20:37:05 +00:00
explicit PolygonFromColumnParser(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-18 19:51:19 +00:00
, offsets(typeid_cast<const ColumnArray &>(*col_).getOffsets())
, ring_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:58:34 +00:00
{}
2021-02-18 19:51:19 +00:00
std::vector<Polygon<Point>> parse(size_t shift, size_t /*size*/) const
2020-06-07 14:58:34 +00:00
{
2021-02-18 19:51:19 +00:00
size_t prev_offset = shift;
std::vector<Polygon<Point>> answer(offsets.size());
size_t iter = 0;
2020-06-07 14:58:34 +00:00
2021-02-18 19:51:19 +00:00
for (size_t offset : offsets)
2020-06-07 14:58:34 +00:00
{
2021-02-18 19:51:19 +00:00
offset += shift;
auto tmp = ring_parser.parse(prev_offset, offset - prev_offset);
/// FIXME: other rings are holes in first
answer[iter].outer() = tmp[0];
prev_offset = offset;
++iter;
2020-06-07 14:58:34 +00:00
}
2021-02-18 19:51:19 +00:00
return answer;
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;
2020-06-07 14:58:34 +00:00
const IColumn::Offsets & offsets;
2021-01-19 14:52:53 +00:00
const RingFromColumnParser<Point> ring_parser;
2020-06-07 14:58:34 +00:00
};
2021-01-19 14:52:53 +00:00
template<class Point>
2020-06-07 14:58:34 +00:00
class MultiPolygonFromColumnParser
{
public:
2021-02-11 20:37:05 +00:00
explicit MultiPolygonFromColumnParser(ColumnPtr col_)
2021-01-19 22:07:10 +00:00
: col(col_)
2021-02-18 19:51:19 +00:00
, offsets(typeid_cast<const ColumnArray &>(*col_).getOffsets())
, polygon_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
2020-06-07 14:58:34 +00:00
{}
2021-02-18 19:51:19 +00:00
std::vector<MultiPolygon<Point>> parse(size_t shift, size_t /*size*/) const
2020-06-07 14:58:34 +00:00
{
2021-02-18 19:51:19 +00:00
size_t prev_offset = shift;
std::vector<MultiPolygon<Point>> answer;
answer.resize(offsets.size());
2021-02-15 19:22:13 +00:00
2021-02-18 19:51:19 +00:00
size_t iter = 0;
2020-06-07 14:58:34 +00:00
2021-02-18 19:51:19 +00:00
for (size_t offset : offsets)
2020-06-07 14:58:34 +00:00
{
2021-02-18 19:51:19 +00:00
offset += shift;
auto polygons = polygon_parser.parse(prev_offset, offset - prev_offset);
answer[iter].swap(polygons);
prev_offset = offset;
++iter;
2020-06-07 14:58:34 +00:00
}
2021-02-18 19:51:19 +00:00
return answer;
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;
2020-06-07 14:58:34 +00:00
const IColumn::Offsets & offsets;
2021-01-19 14:52:53 +00:00
const PolygonFromColumnParser<Point> polygon_parser;
2020-06-07 14:58:34 +00:00
};
2021-01-19 14:52:53 +00:00
template <typename Point>
using GeometryFromColumnParser = boost::variant<
PointFromColumnParser<Point>,
RingFromColumnParser<Point>,
PolygonFromColumnParser<Point>,
MultiPolygonFromColumnParser<Point>
2020-06-07 14:58:34 +00:00
>;
2020-06-07 13:42:09 +00:00
2021-02-18 19:51:19 +00:00
// template <typename Point>
// Geometry<Point> createContainer(const GeometryFromColumnParser<Point> & parser);
2021-01-18 23:51:34 +00:00
2021-02-18 19:51:19 +00:00
// template <typename Point>
// void get(const GeometryFromColumnParser<Point> & parser, Geometry<Point> & container, size_t i);
2021-01-18 23:51:34 +00:00
2021-02-18 19:51:19 +00:00
// template <typename Point>
// GeometryFromColumnParser<Point> makeGeometryFromColumnParser(const ColumnWithTypeAndName & col);
2021-01-18 23:51:34 +00:00
2021-02-15 19:22:13 +00:00
2021-02-18 19:51:19 +00:00
// extern template Geometry<CartesianPoint> createContainer(const GeometryFromColumnParser<CartesianPoint> & parser);
// extern template Geometry<GeographicPoint> createContainer(const GeometryFromColumnParser<GeographicPoint> & parser);
// extern template void get(const GeometryFromColumnParser<CartesianPoint> & parser, Geometry<CartesianPoint> & container, size_t i);
// extern template void get(const GeometryFromColumnParser<GeographicPoint> & parser, Geometry<GeographicPoint> & container, size_t i);
// extern template GeometryFromColumnParser<CartesianPoint> makeGeometryFromColumnParser(const ColumnWithTypeAndName & col);
// extern template GeometryFromColumnParser<GeographicPoint> makeGeometryFromColumnParser(const ColumnWithTypeAndName & col);
2020-06-07 16:04:35 +00:00
2021-01-19 14:52:53 +00:00
/// To serialize Geographic or Cartesian point (a pair of numbers in both cases).
template <typename Point>
class PointSerializerVisitor : public boost::static_visitor<void>
2020-06-14 17:04:10 +00:00
{
2020-06-14 17:37:15 +00:00
public:
2021-01-19 14:52:53 +00:00
PointSerializerVisitor()
: first(ColumnFloat64::create())
, second(ColumnFloat64::create())
2020-06-14 17:04:10 +00:00
{}
2021-02-11 20:37:05 +00:00
explicit PointSerializerVisitor(size_t n)
2021-01-19 14:52:53 +00:00
: first(ColumnFloat64::create(n))
, second(ColumnFloat64::create(n))
2020-06-14 17:04:10 +00:00
{}
2021-01-19 14:52:53 +00:00
void operator()(const Point & point)
2020-06-14 17:04:10 +00:00
{
2021-01-19 14:52:53 +00:00
first->insertValue(point.template get<0>());
second->insertValue(point.template get<1>());
2020-06-14 17:04:10 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Ring<Point> & ring)
2020-06-14 17:04:10 +00:00
{
2021-01-19 17:16:10 +00:00
if (ring.size() != 1)
2020-06-14 17:04:10 +00:00
throw Exception("Unable to write ring of size " + toString(ring.size()) + " != 1 to point column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:04:10 +00:00
(*this)(ring[0]);
}
2021-01-19 14:52:53 +00:00
void operator()(const Polygon<Point> & polygon)
2020-06-14 17:04:10 +00:00
{
2021-01-19 17:16:10 +00:00
if (polygon.inners().size() != 0)
2020-06-14 17:04:10 +00:00
throw Exception("Unable to write polygon with holes to point column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:04:10 +00:00
(*this)(polygon.outer());
}
2021-01-19 14:52:53 +00:00
void operator()(const MultiPolygon<Point> & multi_polygon)
2020-06-14 17:04:10 +00:00
{
2021-01-19 17:16:10 +00:00
if (multi_polygon.size() != 1)
2020-06-14 17:04:10 +00:00
throw Exception("Unable to write multi-polygon of size " + toString(multi_polygon.size()) + " != 1 to point column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:04:10 +00:00
(*this)(multi_polygon[0]);
}
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;
2020-06-14 17:37:15 +00:00
};
2021-01-19 14:52:53 +00:00
template <typename Point>
class RingSerializerVisitor : public boost::static_visitor<void>
2020-06-14 17:37:15 +00:00
{
public:
2021-01-19 14:52:53 +00:00
RingSerializerVisitor()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
2021-02-11 20:37:05 +00:00
explicit RingSerializerVisitor(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
2021-01-19 14:52:53 +00:00
void operator()(const Point & point)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
point_serializer(point);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size += ring.size();
offsets->insertValue(size);
for (const auto & point : ring)
{
2021-02-12 10:02:44 +00:00
point_serializer(point);
2020-06-14 17:37:15 +00:00
}
}
2021-01-19 14:52:53 +00:00
void operator()(const Polygon<Point> & polygon)
2020-06-14 17:37:15 +00:00
{
2021-01-19 17:16:10 +00:00
if (polygon.inners().size() != 0)
2020-06-14 17:37:15 +00:00
throw Exception("Unable to write polygon with holes to ring column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:37:15 +00:00
(*this)(polygon.outer());
}
2021-01-19 14:52:53 +00:00
void operator()(const MultiPolygon<Point> & multi_polygon)
2020-06-14 17:37:15 +00:00
{
2021-01-19 17:16:10 +00:00
if (multi_polygon.size() != 1)
2020-06-14 17:37:15 +00:00
throw Exception("Unable to write multi-polygon of size " + toString(multi_polygon.size()) + " != 1 to ring column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:37:15 +00:00
(*this)(multi_polygon[0]);
}
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;
2021-02-11 20:37:05 +00:00
PointSerializerVisitor<Point> point_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2021-01-19 14:52:53 +00:00
template <typename Point>
class PolygonSerializerVisitor : public boost::static_visitor<void>
2020-06-14 17:37:15 +00:00
{
public:
2021-01-19 14:52:53 +00:00
PolygonSerializerVisitor()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
2021-02-11 20:37:05 +00:00
explicit PolygonSerializerVisitor(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
2021-01-19 14:52:53 +00:00
void operator()(const Point & point)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
ring_serializer(point);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
ring_serializer(ring);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Polygon<Point> & polygon)
2020-06-14 17:37:15 +00:00
{
size += 1 + polygon.inners().size();
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
ring_serializer(polygon.outer());
2020-06-14 17:37:15 +00:00
for (const auto & ring : polygon.inners())
{
2021-02-12 10:02:44 +00:00
ring_serializer(ring);
2020-06-14 17:37:15 +00:00
}
}
2021-01-19 14:52:53 +00:00
void operator()(const MultiPolygon<Point> & multi_polygon)
2020-06-14 17:37:15 +00:00
{
2021-01-19 17:16:10 +00:00
if (multi_polygon.size() != 1)
2020-06-14 17:37:15 +00:00
throw Exception("Unable to write multi-polygon of size " + toString(multi_polygon.size()) + " != 1 to polygon column", ErrorCodes::BAD_ARGUMENTS);
2021-01-19 17:16:10 +00:00
2020-06-14 17:37:15 +00:00
(*this)(multi_polygon[0]);
}
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;
2021-02-11 20:37:05 +00:00
RingSerializerVisitor<Point> ring_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2021-01-19 14:52:53 +00:00
template <typename Point>
class MultiPolygonSerializerVisitor : public boost::static_visitor<void>
2020-06-14 17:37:15 +00:00
{
public:
2021-01-19 14:52:53 +00:00
MultiPolygonSerializerVisitor()
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create())
{}
2021-02-11 20:37:05 +00:00
explicit MultiPolygonSerializerVisitor(size_t n)
2020-06-14 17:37:15 +00:00
: offsets(ColumnUInt64::create(n))
{}
2021-01-19 14:52:53 +00:00
void operator()(const Point & point)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
polygon_serializer(point);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Ring<Point> & ring)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
polygon_serializer(ring);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(const Polygon<Point> & polygon)
2020-06-14 17:37:15 +00:00
{
size++;
offsets->insertValue(size);
2021-02-12 10:02:44 +00:00
polygon_serializer(polygon);
2020-06-14 17:37:15 +00:00
}
2021-01-19 14:52:53 +00:00
void operator()(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)
{
2021-02-12 10:02:44 +00:00
polygon_serializer(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;
2021-02-11 20:37:05 +00:00
PolygonSerializerVisitor<Point> polygon_serializer;
2020-06-14 17:37:15 +00:00
ColumnUInt64::MutablePtr offsets;
};
2020-06-14 17:04:10 +00:00
template <class Geometry, class Visitor>
class GeometrySerializer
{
public:
void add(const Geometry & geometry)
{
boost::apply_visitor(visitor, geometry);
}
ColumnPtr finalize()
{
return visitor.finalize();
}
private:
Visitor visitor;
2020-06-14 17:37:15 +00:00
};
2020-06-14 17:04:10 +00:00
2021-01-19 14:52:53 +00:00
template <typename Point>
using PointSerializer = GeometrySerializer<Geometry<Point>, PointSerializerVisitor<Point>>;
template <typename Point>
using RingSerializer = GeometrySerializer<Geometry<Point>, RingSerializerVisitor<Point>>;
template <typename Point>
using PolygonSerializer = GeometrySerializer<Geometry<Point>, PolygonSerializerVisitor<Point>>;
template <typename Point>
using MultiPolygonSerializer = GeometrySerializer<Geometry<Point>, MultiPolygonSerializerVisitor<Point>>;
2020-06-14 17:04:10 +00:00
2021-02-15 19:22:13 +00:00
template <typename Point, template<typename> typename Desired>
void checkColumnTypeOrThrow(const ColumnWithTypeAndName & column);
2020-06-07 12:33:49 +00:00
}