2020-06-07 13:42:09 +00:00
|
|
|
#pragma once
|
|
|
|
|
2020-06-07 12:33:49 +00:00
|
|
|
#include <Core/Types.h>
|
|
|
|
|
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>
|
2021-02-19 22:26:33 +00:00
|
|
|
#include <DataTypes/DataTypeCustomGeo.h>
|
2021-03-03 16:47:10 +00:00
|
|
|
#include <Functions/geometryTypes.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-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-26 15:29:26 +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-18 23:51:34 +00:00
|
|
|
/**
|
2021-02-26 15:29:26 +00:00
|
|
|
* Class which takes converts Column with type Tuple(Float64, Float64) to a vector of boost point type.
|
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-26 15:29:26 +00:00
|
|
|
struct ColumnToPointsConverter
|
2020-06-07 13:42:09 +00:00
|
|
|
{
|
2021-02-26 15:29:26 +00:00
|
|
|
static std::vector<Point> convert(ColumnPtr col)
|
2020-06-07 14:28:46 +00:00
|
|
|
{
|
2021-02-26 15:29:26 +00:00
|
|
|
const auto * tuple = typeid_cast<const ColumnTuple *>(col.get());
|
|
|
|
const auto & tuple_columns = tuple->getColumns();
|
2020-06-07 14:28:46 +00:00
|
|
|
|
2021-02-26 15:29:26 +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());
|
2021-01-18 23:51:34 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
const auto * first_container = x_data->getData().data();
|
|
|
|
const auto * second_container = y_data->getData().data();
|
2021-02-19 22:26:33 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
std::vector<Point> answer(col->size());
|
2020-06-07 13:42:09 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
for (size_t i = 0; i < col->size(); ++i)
|
|
|
|
{
|
|
|
|
const Float64 first = first_container[i];
|
|
|
|
const Float64 second = second_container[i];
|
2021-02-18 19:51:19 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
if (isNaN(first) || isNaN(second))
|
|
|
|
throw Exception("Point's component must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-06-07 14:28:46 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
if (isinf(first) || isinf(second))
|
|
|
|
throw Exception("Point's component must not be infinite", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-06-07 14:28:46 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
answer[i] = Point(first, second);
|
|
|
|
}
|
|
|
|
|
|
|
|
return answer;
|
|
|
|
}
|
2020-06-07 14:28:46 +00:00
|
|
|
};
|
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
|
|
|
|
template <typename Point>
|
|
|
|
struct ColumnToRingsConverter
|
2020-06-07 14:58:34 +00:00
|
|
|
{
|
2021-02-26 15:29:26 +00:00
|
|
|
static std::vector<Ring<Point>> convert(ColumnPtr col)
|
2020-06-07 14:58:34 +00:00
|
|
|
{
|
2021-02-26 15:29:26 +00:00
|
|
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
|
|
|
size_t prev_offset = 0;
|
|
|
|
std::vector<Ring<Point>> answer;
|
|
|
|
answer.reserve(offsets.size());
|
|
|
|
auto tmp = ColumnToPointsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
|
|
|
for (size_t offset : offsets)
|
|
|
|
{
|
|
|
|
answer.emplace_back(tmp.begin() + prev_offset, tmp.begin() + offset);
|
|
|
|
prev_offset = offset;
|
|
|
|
}
|
|
|
|
return answer;
|
2020-06-07 14:58:34 +00:00
|
|
|
}
|
2021-02-26 15:29:26 +00:00
|
|
|
};
|
2020-06-07 14:58:34 +00:00
|
|
|
|
2021-02-19 18:09:38 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
template <typename Point>
|
|
|
|
struct ColumnToPolygonsConverter
|
|
|
|
{
|
|
|
|
static std::vector<Polygon<Point>> convert(ColumnPtr col)
|
|
|
|
{
|
|
|
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
|
|
|
std::vector<Polygon<Point>> answer(offsets.size());
|
|
|
|
auto all_rings = ColumnToRingsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
|
|
|
|
|
|
|
size_t prev_offset = 0;
|
|
|
|
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
|
|
|
{
|
|
|
|
const auto current_array_size = offsets[iter] - prev_offset;
|
|
|
|
answer[iter].outer() = std::move(all_rings[prev_offset]);
|
|
|
|
answer[iter].inners().reserve(current_array_size);
|
|
|
|
for (size_t inner_holes = prev_offset + 1; inner_holes < offsets[iter]; ++inner_holes)
|
|
|
|
answer[iter].inners().emplace_back(std::move(all_rings[inner_holes]));
|
|
|
|
prev_offset = offsets[iter];
|
|
|
|
}
|
2021-02-19 18:09:38 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
return answer;
|
|
|
|
}
|
2020-06-07 14:58:34 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
template <typename Point>
|
|
|
|
struct ColumnToMultiPolygonsConverter
|
|
|
|
{
|
|
|
|
static std::vector<MultiPolygon<Point>> convert(ColumnPtr col)
|
|
|
|
{
|
|
|
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
|
|
|
size_t prev_offset = 0;
|
|
|
|
std::vector<MultiPolygon<Point>> answer(offsets.size());
|
2020-06-07 14:58:34 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
auto all_polygons = ColumnToPolygonsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
2020-06-07 14:58:34 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
|
|
|
{
|
|
|
|
for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter)
|
|
|
|
answer[iter].emplace_back(std::move(all_polygons[polygon_iter]));
|
|
|
|
prev_offset = offsets[iter];
|
|
|
|
}
|
2021-01-18 23:51:34 +00:00
|
|
|
|
2021-02-26 15:29:26 +00:00
|
|
|
return answer;
|
|
|
|
}
|
|
|
|
};
|
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>
|
2021-02-19 22:26:33 +00:00
|
|
|
class PointSerializer
|
2020-06-14 17:04:10 +00:00
|
|
|
{
|
2020-06-14 17:37:15 +00:00
|
|
|
public:
|
2021-02-19 22:26:33 +00:00
|
|
|
PointSerializer()
|
2021-01-19 14:52:53 +00:00
|
|
|
: first(ColumnFloat64::create())
|
|
|
|
, second(ColumnFloat64::create())
|
2021-02-19 22:26:33 +00:00
|
|
|
, first_container(first->getData())
|
|
|
|
, second_container(second->getData())
|
2020-06-14 17:04:10 +00:00
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
explicit PointSerializer(size_t n)
|
2021-01-19 14:52:53 +00:00
|
|
|
: first(ColumnFloat64::create(n))
|
|
|
|
, second(ColumnFloat64::create(n))
|
2021-02-19 22:26:33 +00:00
|
|
|
, first_container(first->getData())
|
|
|
|
, second_container(second->getData())
|
2020-06-14 17:04:10 +00:00
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const Point & point)
|
2020-06-14 17:04:10 +00:00
|
|
|
{
|
2021-02-19 22:26:33 +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;
|
2021-02-19 22:26:33 +00:00
|
|
|
|
|
|
|
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>
|
2021-02-19 22:26:33 +00:00
|
|
|
class RingSerializer
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-02-19 22:26:33 +00:00
|
|
|
RingSerializer()
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create())
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
explicit RingSerializer(size_t n)
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create(n))
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
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)
|
2021-02-19 22:26:33 +00:00
|
|
|
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;
|
2021-02-19 22:26:33 +00:00
|
|
|
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>
|
2021-02-19 22:26:33 +00:00
|
|
|
class PolygonSerializer
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-02-19 22:26:33 +00:00
|
|
|
PolygonSerializer()
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create())
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
explicit PolygonSerializer(size_t n)
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create(n))
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const Ring<Point> & ring)
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
size++;
|
|
|
|
offsets->insertValue(size);
|
2021-02-19 22:26:33 +00:00
|
|
|
ring_serializer.add(ring);
|
2020-06-14 17:37:15 +00:00
|
|
|
}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const Polygon<Point> & polygon)
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
2021-02-26 15:29:26 +00:00
|
|
|
/// Outer ring + all inner rings (holes).
|
2020-06-14 17:37:15 +00:00
|
|
|
size += 1 + polygon.inners().size();
|
|
|
|
offsets->insertValue(size);
|
2021-02-19 22:26:33 +00:00
|
|
|
ring_serializer.add(polygon.outer());
|
2020-06-14 17:37:15 +00:00
|
|
|
for (const auto & ring : polygon.inners())
|
2021-02-19 22:26:33 +00:00
|
|
|
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;
|
2021-02-19 22:26:33 +00:00
|
|
|
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>
|
2021-02-19 22:26:33 +00:00
|
|
|
class MultiPolygonSerializer
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-02-19 22:26:33 +00:00
|
|
|
MultiPolygonSerializer()
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create())
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
explicit MultiPolygonSerializer(size_t n)
|
2020-06-14 17:37:15 +00:00
|
|
|
: offsets(ColumnUInt64::create(n))
|
|
|
|
{}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const Ring<Point> & ring)
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
size++;
|
|
|
|
offsets->insertValue(size);
|
2021-02-19 22:26:33 +00:00
|
|
|
polygon_serializer.add(ring);
|
2020-06-14 17:37:15 +00:00
|
|
|
}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const Polygon<Point> & polygon)
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
|
|
|
size++;
|
|
|
|
offsets->insertValue(size);
|
2021-02-19 22:26:33 +00:00
|
|
|
polygon_serializer.add(polygon);
|
2020-06-14 17:37:15 +00:00
|
|
|
}
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
void add(const MultiPolygon<Point> & multi_polygon)
|
2020-06-14 17:37:15 +00:00
|
|
|
{
|
2020-06-21 15:22:00 +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-19 22:26:33 +00:00
|
|
|
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;
|
2021-02-19 22:26:33 +00:00
|
|
|
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;
|
|
|
|
};
|
|
|
|
|
2021-02-19 22:26:33 +00:00
|
|
|
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))
|
2021-02-26 15:29:26 +00:00
|
|
|
return f(ConverterType<ColumnToPointsConverter<Point>>());
|
2021-02-20 17:44:18 +00:00
|
|
|
else if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
|
2021-02-26 15:29:26 +00:00
|
|
|
return f(ConverterType<ColumnToRingsConverter<Point>>());
|
2021-02-20 17:44:18 +00:00
|
|
|
else if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
2021-02-26 15:29:26 +00:00
|
|
|
return f(ConverterType<ColumnToPolygonsConverter<Point>>());
|
2021-02-20 17:44:18 +00:00
|
|
|
else if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
|
2021-02-26 15:29:26 +00:00
|
|
|
return f(ConverterType<ColumnToMultiPolygonsConverter<Point>>());
|
2021-02-19 22:26:33 +00:00
|
|
|
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
|
|
|
|
2021-02-19 22:26:33 +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
|
|
|
|
2021-02-19 22:26:33 +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 22:26:33 +00:00
|
|
|
});
|
|
|
|
});
|
|
|
|
}
|
2021-02-19 18:09:38 +00:00
|
|
|
|
2020-06-07 12:33:49 +00:00
|
|
|
}
|