mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
review fix
This commit is contained in:
parent
0e1b2d8fcf
commit
a6c8d50cc2
@ -1,137 +0,0 @@
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Point> PointFromColumnConverter<Point>::convertImpl(size_t shift, size_t count) const
|
||||
{
|
||||
const auto * tuple = typeid_cast<const ColumnTuple *>(col.get());
|
||||
const auto & tuple_columns = tuple->getColumns();
|
||||
|
||||
const auto * x_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[0].get());
|
||||
const auto * y_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[1].get());
|
||||
|
||||
const auto * first_container = x_data->getData().data() + shift;
|
||||
const auto * second_container = y_data->getData().data() + shift;
|
||||
|
||||
std::vector<Point> answer(count);
|
||||
|
||||
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] = Point(first, second);
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Ring<Point>> RingFromColumnConverter<Point>::convert() const
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
size_t prev_offset = 0;
|
||||
std::vector<Ring<Point>> answer;
|
||||
answer.reserve(offsets.size());
|
||||
for (size_t offset : offsets)
|
||||
{
|
||||
auto tmp = point_converter.convertImpl(prev_offset, offset - prev_offset);
|
||||
answer.emplace_back(tmp.begin(), tmp.end());
|
||||
prev_offset = offset;
|
||||
}
|
||||
return answer;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Polygon<Point>> PolygonFromColumnConverter<Point>::convert() const
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
std::vector<Polygon<Point>> answer(offsets.size());
|
||||
auto all_rings = ring_converter.convert();
|
||||
|
||||
auto 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];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
|
||||
|
||||
template <typename Point>
|
||||
std::vector<MultiPolygon<Point>> MultiPolygonFromColumnConverter<Point>::convert() const
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
size_t prev_offset = 0;
|
||||
std::vector<MultiPolygon<Point>> answer(offsets.size());
|
||||
|
||||
auto all_polygons = polygon_converter.convert();
|
||||
|
||||
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];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
|
||||
|
||||
template class PointFromColumnConverter<CartesianPoint>;
|
||||
template class PointFromColumnConverter<SphericalPoint>;
|
||||
template class RingFromColumnConverter<CartesianPoint>;
|
||||
template class RingFromColumnConverter<SphericalPoint>;
|
||||
template class PolygonFromColumnConverter<CartesianPoint>;
|
||||
template class PolygonFromColumnConverter<SphericalPoint>;
|
||||
template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||
template class MultiPolygonFromColumnConverter<SphericalPoint>;
|
||||
|
||||
template <typename Point, template<typename> typename Desired>
|
||||
void checkColumnTypeOrThrow(const ColumnWithTypeAndName & column)
|
||||
{
|
||||
DataTypePtr desired_type;
|
||||
if constexpr (std::is_same_v<Desired<Point>, Ring<Point>>)
|
||||
desired_type = DataTypeCustomRingSerialization::nestedDataType();
|
||||
else if constexpr (std::is_same_v<Desired<Point>, Polygon<Point>>)
|
||||
desired_type = DataTypeCustomPolygonSerialization::nestedDataType();
|
||||
else if constexpr (std::is_same_v<Desired<Point>, MultiPolygon<Point>>)
|
||||
desired_type = DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
else
|
||||
throw Exception("Unexpected Desired type.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!desired_type->equals(*column.type))
|
||||
throw Exception(fmt::format("Expected type {} (MultiPolygon), but got {}", desired_type->getName(), column.type->getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
template void checkColumnTypeOrThrow<CartesianPoint, Ring>(const ColumnWithTypeAndName &);
|
||||
template void checkColumnTypeOrThrow<CartesianPoint, Polygon>(const ColumnWithTypeAndName &);
|
||||
template void checkColumnTypeOrThrow<CartesianPoint, MultiPolygon>(const ColumnWithTypeAndName &);
|
||||
template void checkColumnTypeOrThrow<SphericalPoint, Ring>(const ColumnWithTypeAndName &);
|
||||
template void checkColumnTypeOrThrow<SphericalPoint, Polygon>(const ColumnWithTypeAndName &);
|
||||
template void checkColumnTypeOrThrow<SphericalPoint, MultiPolygon>(const ColumnWithTypeAndName &);
|
||||
|
||||
}
|
@ -3,7 +3,6 @@
|
||||
#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>
|
||||
@ -27,6 +26,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
@ -38,128 +38,123 @@ using Polygon = boost::geometry::model::polygon<Point>;
|
||||
template <typename Point>
|
||||
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon<Point>>;
|
||||
|
||||
template <typename Point>
|
||||
using Geometry = boost::variant<Point, Ring<Point>, Polygon<Point>, MultiPolygon<Point>>;
|
||||
|
||||
template <typename Point>
|
||||
using Figure = boost::variant<Ring<Point>, Polygon<Point>, MultiPolygon<Point>>;
|
||||
|
||||
|
||||
using CartesianPoint = boost::geometry::model::d2::point_xy<Float64>;
|
||||
using CartesianRing = Ring<CartesianPoint>;
|
||||
using CartesianPolygon = Polygon<CartesianPoint>;
|
||||
using CartesianMultiPolygon = MultiPolygon<CartesianPoint>;
|
||||
using CartesianGeometry = Geometry<CartesianPoint>;
|
||||
|
||||
// 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>;
|
||||
|
||||
|
||||
template<class Point>
|
||||
class RingFromColumnConverter;
|
||||
|
||||
template<class Point>
|
||||
class PolygonFromColumnConverter;
|
||||
|
||||
template<class Point>
|
||||
class MultiPolygonFromColumnConverter;
|
||||
|
||||
/**
|
||||
* Class which takes some boost type and returns a pair of numbers.
|
||||
* Class which takes converts Column with type Tuple(Float64, Float64) to a vector of boost point type.
|
||||
* They are (x,y) in case of cartesian coordinated and (lon,lat) in case of Spherical.
|
||||
*/
|
||||
template <typename Point>
|
||||
class PointFromColumnConverter
|
||||
struct ColumnToPointsConverter
|
||||
{
|
||||
public:
|
||||
explicit PointFromColumnConverter(ColumnPtr col_) : col(col_)
|
||||
static std::vector<Point> convert(ColumnPtr col)
|
||||
{
|
||||
const auto * tuple = typeid_cast<const ColumnTuple *>(col.get());
|
||||
const auto & tuple_columns = tuple->getColumns();
|
||||
|
||||
const auto * x_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[0].get());
|
||||
const auto * y_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[1].get());
|
||||
|
||||
const auto * first_container = x_data->getData().data();
|
||||
const auto * second_container = y_data->getData().data();
|
||||
|
||||
std::vector<Point> answer(col->size());
|
||||
|
||||
for (size_t i = 0; i < col->size(); ++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] = Point(first, second);
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
|
||||
std::vector<Point> convert() const
|
||||
{
|
||||
return convertImpl(0, col->size());
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<Point> convertImpl(size_t shift, size_t count) const;
|
||||
|
||||
friend class RingFromColumnConverter<Point>;
|
||||
ColumnPtr col{nullptr};
|
||||
};
|
||||
|
||||
|
||||
template<class Point>
|
||||
class RingFromColumnConverter
|
||||
template <typename Point>
|
||||
struct ColumnToRingsConverter
|
||||
{
|
||||
public:
|
||||
explicit RingFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, point_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
static std::vector<Ring<Point>> convert(ColumnPtr col)
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
std::vector<Ring<Point>> convert() const;
|
||||
|
||||
private:
|
||||
friend class PointFromColumnConverter<Point>;
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const PointFromColumnConverter<Point> point_converter{};
|
||||
};
|
||||
|
||||
template<class Point>
|
||||
class PolygonFromColumnConverter
|
||||
|
||||
template <typename Point>
|
||||
struct ColumnToPolygonsConverter
|
||||
{
|
||||
public:
|
||||
explicit PolygonFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, ring_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
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];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
|
||||
std::vector<Polygon<Point>> convert() const;
|
||||
|
||||
private:
|
||||
friend class MultiPolygonFromColumnConverter<Point>;
|
||||
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const RingFromColumnConverter<Point> ring_converter{};
|
||||
};
|
||||
|
||||
template<class Point>
|
||||
class MultiPolygonFromColumnConverter
|
||||
|
||||
template <typename Point>
|
||||
struct ColumnToMultiPolygonsConverter
|
||||
{
|
||||
public:
|
||||
explicit MultiPolygonFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, polygon_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
{}
|
||||
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());
|
||||
|
||||
std::vector<MultiPolygon<Point>> convert() const;
|
||||
auto all_polygons = ColumnToPolygonsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
||||
|
||||
private:
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const PolygonFromColumnConverter<Point> polygon_converter{};
|
||||
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];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
extern template class PointFromColumnConverter<CartesianPoint>;
|
||||
extern template class PointFromColumnConverter<SphericalPoint>;
|
||||
extern template class RingFromColumnConverter<CartesianPoint>;
|
||||
extern template class RingFromColumnConverter<SphericalPoint>;
|
||||
extern template class PolygonFromColumnConverter<CartesianPoint>;
|
||||
extern template class PolygonFromColumnConverter<SphericalPoint>;
|
||||
extern template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||
extern template class MultiPolygonFromColumnConverter<SphericalPoint>;
|
||||
|
||||
|
||||
/// To serialize Spherical or Cartesian point (a pair of numbers in both cases).
|
||||
template <typename Point>
|
||||
class PointSerializer
|
||||
@ -256,6 +251,7 @@ public:
|
||||
|
||||
void add(const Polygon<Point> & polygon)
|
||||
{
|
||||
/// Outer ring + all inner rings (holes).
|
||||
size += 1 + polygon.inners().size();
|
||||
offsets->insertValue(size);
|
||||
ring_serializer.add(polygon.outer());
|
||||
@ -334,13 +330,13 @@ static void callOnGeometryDataType(DataTypePtr type, F && f)
|
||||
{
|
||||
/// There is no Point type, because for most of geometry functions it is useless.
|
||||
if (DataTypeCustomPointSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<PointFromColumnConverter<Point>>());
|
||||
return f(ConverterType<ColumnToPointsConverter<Point>>());
|
||||
else if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<RingFromColumnConverter<Point>>());
|
||||
return f(ConverterType<ColumnToRingsConverter<Point>>());
|
||||
else if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<PolygonFromColumnConverter<Point>>());
|
||||
return f(ConverterType<ColumnToPolygonsConverter<Point>>());
|
||||
else if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<MultiPolygonFromColumnConverter<Point>>());
|
||||
return f(ConverterType<ColumnToMultiPolygonsConverter<Point>>());
|
||||
throw Exception(fmt::format("Unknown geometry type {}", type->getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
|
@ -427,8 +427,8 @@ private:
|
||||
out_container.reserve(end - begin);
|
||||
for (size_t i = begin; i < end; ++i)
|
||||
{
|
||||
int64_t result = 0;
|
||||
if (common::mulOverflow(static_cast<int64_t>(x_data[i]), static_cast<int64_t>(y_data[i]), result))
|
||||
Int64 result = 0;
|
||||
if (common::mulOverflow(static_cast<Int64>(x_data[i]), static_cast<Int64>(y_data[i]), result))
|
||||
throw Exception("The coordinates of the point are such that subsequent calculations cannot be performed correctly. " \
|
||||
"Most likely they are very large in modulus.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
|
@ -69,35 +69,11 @@ public:
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, Converter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, Converter>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto geometries = converter.convert();
|
||||
|
||||
if constexpr (std::is_same_v<PolygonFromColumnConverter<Point>, Converter>) {
|
||||
for (auto & polygon : geometries) {
|
||||
std::cout << "OUTER" << std::endl;
|
||||
for (auto point : polygon.outer()) {
|
||||
if constexpr (std::is_same_v<Point, CartesianPoint>) {
|
||||
std::cout << point.x() << ' ' << point.y() << std::endl;
|
||||
} else {
|
||||
std::cout << point.template get<0>() << ' ' << point.template get<1>() << std::endl;
|
||||
}
|
||||
}
|
||||
std::cout << "INNER" << std::endl;
|
||||
for (auto & inner : polygon.inners()) {
|
||||
for (auto point : inner) {
|
||||
if constexpr (std::is_same_v<Point, CartesianPoint>) {
|
||||
std::cout << point.x() << ' ' << point.y() << std::endl;
|
||||
} else {
|
||||
std::cout << point.template get<0>() << ' ' << point.template get<1>() << std::endl;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::area(geometries[i]));
|
||||
|
@ -67,12 +67,11 @@ public:
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
if constexpr (std::is_same_v<Converter, PointFromColumnConverter<Point>>)
|
||||
if constexpr (std::is_same_v<Converter, ColumnToPointsConverter<Point>>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be a Point", getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
else
|
||||
{
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto geometries = converter.convert();
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
|
@ -69,12 +69,11 @@ public:
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, Converter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, Converter>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto geometries = converter.convert();
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::perimeter(geometries[i]));
|
||||
|
@ -74,12 +74,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
|
@ -73,12 +73,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
|
@ -72,12 +72,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
|
@ -71,12 +71,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
|
@ -71,12 +71,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
|
@ -75,12 +75,12 @@ public:
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<PointFromColumnConverter<Point>, LeftConverter> || std::is_same_v<PointFromColumnConverter<Point>, RightConverter>)
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
|
@ -16,12 +16,20 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <class DataType, class Geometry, class Serializer>
|
||||
|
||||
template <class DataType, class Geometry, class Serializer, class NameHolder>
|
||||
class FunctionReadWkt : public IFunction
|
||||
{
|
||||
public:
|
||||
explicit FunctionReadWkt() = default;
|
||||
|
||||
static constexpr const char * name = NameHolder::name;
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
@ -59,71 +67,39 @@ public:
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionReadWktPoint : public FunctionReadWkt<DataTypeCustomPointSerialization, CartesianPoint, PointSerializer<CartesianPoint>>
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "readWktPoint";
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReadWktPoint>();
|
||||
return std::make_shared<FunctionReadWkt<DataType, Geometry, Serializer, NameHolder>>();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionReadWktRing : public FunctionReadWkt<DataTypeCustomRingSerialization, CartesianRing, RingSerializer<CartesianPoint>>
|
||||
struct ReadWktPointNameHolder
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "readWktRing";
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReadWktRing>();
|
||||
}
|
||||
static constexpr const char * name = "readWktPoint";
|
||||
};
|
||||
|
||||
class FunctionReadWktPolygon : public FunctionReadWkt<DataTypeCustomPolygonSerialization, CartesianPolygon, PolygonSerializer<CartesianPoint>>
|
||||
struct ReadWktRingNameHolder
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "readWktPolygon";
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReadWktPolygon>();
|
||||
}
|
||||
static constexpr const char * name = "readWktRing";
|
||||
};
|
||||
|
||||
class FunctionReadWktMultiPolygon : public FunctionReadWkt<DataTypeCustomMultiPolygonSerialization, CartesianMultiPolygon, MultiPolygonSerializer<CartesianPoint>>
|
||||
struct ReadWktPolygonNameHolder
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "readWktMultiPolygon";
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReadWktMultiPolygon>();
|
||||
}
|
||||
static constexpr const char * name = "readWktPolygon";
|
||||
};
|
||||
|
||||
struct ReadWktMultiPolygonNameHolder
|
||||
{
|
||||
static constexpr const char * name = "readWktMultiPolygon";
|
||||
};
|
||||
|
||||
void registerFunctionReadWkt(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionReadWktPoint>();
|
||||
factory.registerFunction<FunctionReadWktRing>();
|
||||
factory.registerFunction<FunctionReadWktPolygon>();
|
||||
factory.registerFunction<FunctionReadWktMultiPolygon>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomPointSerialization, CartesianPoint, PointSerializer<CartesianPoint>, ReadWktPointNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomRingSerialization, CartesianRing, RingSerializer<CartesianPoint>, ReadWktRingNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomPolygonSerialization, CartesianPolygon, PolygonSerializer<CartesianPoint>, ReadWktPolygonNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomMultiPolygonSerialization, CartesianMultiPolygon, MultiPolygonSerializer<CartesianPoint>, ReadWktMultiPolygonNameHolder>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -64,21 +64,18 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
|
||||
auto res_column = ColumnString::create();
|
||||
bool has_style = arguments.size() > 1;
|
||||
ColumnPtr style;
|
||||
if (has_style)
|
||||
style = arguments[1].column;
|
||||
|
||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = converter.convert();
|
||||
|
||||
bool has_style = arguments.size() > 1;
|
||||
ColumnPtr style;
|
||||
if (has_style)
|
||||
style = arguments[1].column;
|
||||
auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
|
@ -45,8 +45,7 @@ public:
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = converter.convert();
|
||||
auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
|
@ -268,7 +268,6 @@ SRCS(
|
||||
geohashDecode.cpp
|
||||
geohashEncode.cpp
|
||||
geohashesInBox.cpp
|
||||
geometryConverters.cpp
|
||||
getMacro.cpp
|
||||
getScalar.cpp
|
||||
getSetting.cpp
|
||||
|
@ -45,7 +45,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
WriteBufferFromVector(VectorType & vector_)
|
||||
explicit WriteBufferFromVector(VectorType & vector_)
|
||||
: WriteBuffer(reinterpret_cast<Position>(vector_.data()), vector_.size()), vector(vector_)
|
||||
{
|
||||
if (vector.empty())
|
||||
|
@ -1,2 +1,2 @@
|
||||
select polygonsEqualsCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
select polygonsEqualsCartesian([[[(1., 1.),(1., 4.),(4., 4.),(4., 1.)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
select polygonsEqualsCartesian([[[(1., 1.),(1., 4.),(4., 4.),(4., 1.)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
|
@ -2,4 +2,4 @@ select polygonsIntersectionCartesian([[[(0., 0.),(0., 3.),(1., 2.9),(2., 2.6),(2
|
||||
select polygonsIntersectionCartesian([[[(0., 0.),(0., 3.),(1., 2.9),(2., 2.6),(2.6, 2.),(2.9, 1.),(3., 0.),(0., 0.)]]], [[[(3., 3.),(3., 4.),(4., 4.),(4., 3.),(3., 3.)]]]);
|
||||
|
||||
select polygonsIntersectionSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]], [[[(25.0010, 136.9987), (17.7500, 142.5000), (11.3733, 142.5917)]]]);
|
||||
select polygonsIntersectionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]);
|
||||
select polygonsIntersectionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]);
|
||||
|
@ -1 +1 @@
|
||||
select polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (0., 0.)]]]);
|
||||
select polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (0., 0.)]]]);
|
||||
|
@ -1,3 +1,3 @@
|
||||
select polygonAreaCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]);
|
||||
select polygonAreaSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]);
|
||||
SELECT polygonAreaCartesian([]); -- { serverError 36 }
|
||||
SELECT polygonAreaCartesian([]); -- { serverError 36 }
|
||||
|
Loading…
Reference in New Issue
Block a user