mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
return back all functions
This commit is contained in:
parent
f6712479cd
commit
681b0b63f1
@ -1,131 +0,0 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
class FunctionDescribeGeometry : public IFunction
|
||||
{
|
||||
public:
|
||||
explicit FunctionDescribeGeometry() = default;
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
static inline const char * name = "describeGeometry";
|
||||
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (checkAndGetDataType<DataTypeString>(arguments[0].get()) == nullptr)
|
||||
{
|
||||
throw Exception("First argument should be String",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * column_string = checkAndGetColumn<ColumnString>(arguments[0].column.get());
|
||||
|
||||
CartesianPoint point;
|
||||
Ring<CartesianPoint> ring;
|
||||
Polygon<CartesianPoint> polygon;
|
||||
MultiPolygon<CartesianPoint> multipolygon;
|
||||
|
||||
auto result = ColumnUInt8::create();
|
||||
auto & result_array = result->getData();
|
||||
|
||||
result_array.reserve(input_rows_count);
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
const auto & str = column_string->getDataAt(i).toString();
|
||||
|
||||
try
|
||||
{
|
||||
boost::geometry::read_wkt(str, point);
|
||||
result_array.emplace_back(0);
|
||||
continue;
|
||||
}
|
||||
catch (boost::geometry::read_wkt_exception &)
|
||||
{
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
boost::geometry::read_wkt(str, ring);
|
||||
result_array.emplace_back(1);
|
||||
continue;
|
||||
}
|
||||
catch (boost::geometry::read_wkt_exception &)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
try
|
||||
{
|
||||
boost::geometry::read_wkt(str, polygon);
|
||||
result_array.emplace_back(2);
|
||||
continue;
|
||||
}
|
||||
catch (boost::geometry::read_wkt_exception &)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
try
|
||||
{
|
||||
boost::geometry::read_wkt(str, multipolygon);
|
||||
result_array.emplace_back(3);
|
||||
continue;
|
||||
}
|
||||
catch (boost::geometry::read_wkt_exception &)
|
||||
{
|
||||
}
|
||||
|
||||
throw Exception("Unknown geometry format", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionDescribeGeometry>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionDescribeGeometry(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionDescribeGeometry>();
|
||||
}
|
||||
|
||||
}
|
@ -13,7 +13,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Point> PointFromColumnParser<Point>::parseImpl(size_t shift, size_t count) const
|
||||
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();
|
||||
@ -44,7 +44,7 @@ std::vector<Point> PointFromColumnParser<Point>::parseImpl(size_t shift, size_t
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Ring<Point>> RingFromColumnParser<Point>::parse() const
|
||||
std::vector<Ring<Point>> RingFromColumnConverter<Point>::convert() const
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
size_t prev_offset = 0;
|
||||
@ -52,7 +52,7 @@ std::vector<Ring<Point>> RingFromColumnParser<Point>::parse() const
|
||||
answer.reserve(offsets.size());
|
||||
for (size_t offset : offsets)
|
||||
{
|
||||
auto tmp = point_parser.parseImpl(prev_offset, offset - prev_offset);
|
||||
auto tmp = point_converter.convertImpl(prev_offset, offset - prev_offset);
|
||||
answer.emplace_back(tmp.begin(), tmp.end());
|
||||
prev_offset = offset;
|
||||
}
|
||||
@ -60,11 +60,11 @@ std::vector<Ring<Point>> RingFromColumnParser<Point>::parse() const
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
std::vector<Polygon<Point>> PolygonFromColumnParser<Point>::parse() const
|
||||
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_parser.parse();
|
||||
auto all_rings = ring_converter.convert();
|
||||
|
||||
auto prev_offset = 0;
|
||||
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
||||
@ -82,18 +82,18 @@ std::vector<Polygon<Point>> PolygonFromColumnParser<Point>::parse() const
|
||||
|
||||
|
||||
template <typename Point>
|
||||
std::vector<MultiPolygon<Point>> MultiPolygonFromColumnParser<Point>::parse() const
|
||||
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_parser.parse();
|
||||
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]));
|
||||
answer[iter].emplace_back(std::move(all_polygons[polygon_iter]));
|
||||
prev_offset = offsets[iter];
|
||||
}
|
||||
|
||||
@ -101,14 +101,14 @@ std::vector<MultiPolygon<Point>> MultiPolygonFromColumnParser<Point>::parse() co
|
||||
}
|
||||
|
||||
|
||||
template class PointFromColumnParser<CartesianPoint>;
|
||||
template class PointFromColumnParser<GeographicPoint>;
|
||||
template class RingFromColumnParser<CartesianPoint>;
|
||||
template class RingFromColumnParser<GeographicPoint>;
|
||||
template class PolygonFromColumnParser<CartesianPoint>;
|
||||
template class PolygonFromColumnParser<GeographicPoint>;
|
||||
template class MultiPolygonFromColumnParser<CartesianPoint>;
|
||||
template class MultiPolygonFromColumnParser<GeographicPoint>;
|
||||
template class PointFromColumnConverter<CartesianPoint>;
|
||||
template class PointFromColumnConverter<GeographicPoint>;
|
||||
template class RingFromColumnConverter<CartesianPoint>;
|
||||
template class RingFromColumnConverter<GeographicPoint>;
|
||||
template class PolygonFromColumnConverter<CartesianPoint>;
|
||||
template class PolygonFromColumnConverter<GeographicPoint>;
|
||||
template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||
template class MultiPolygonFromColumnConverter<GeographicPoint>;
|
||||
|
||||
template <typename Point, template<typename> typename Desired>
|
||||
void checkColumnTypeOrThrow(const ColumnWithTypeAndName & column)
|
||||
|
@ -60,109 +60,106 @@ using GeographicGeometry = Geometry<GeographicPoint>;
|
||||
|
||||
|
||||
template<class Point>
|
||||
class RingFromColumnParser;
|
||||
class RingFromColumnConverter;
|
||||
|
||||
template<class Point>
|
||||
class PolygonFromColumnParser;
|
||||
class PolygonFromColumnConverter;
|
||||
|
||||
template<class Point>
|
||||
class MultiPolygonFromColumnParser;
|
||||
class MultiPolygonFromColumnConverter;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
template <typename Point>
|
||||
class PointFromColumnParser
|
||||
class PointFromColumnConverter
|
||||
{
|
||||
public:
|
||||
PointFromColumnParser() = default;
|
||||
|
||||
explicit PointFromColumnParser(ColumnPtr col_) : col(col_)
|
||||
explicit PointFromColumnConverter(ColumnPtr col_) : col(col_)
|
||||
{
|
||||
}
|
||||
|
||||
std::vector<Point> parse() const;
|
||||
std::vector<Point> convert() const;
|
||||
|
||||
private:
|
||||
std::vector<Point> parseImpl(size_t shift, size_t count) const;
|
||||
std::vector<Point> convertImpl(size_t shift, size_t count) const;
|
||||
|
||||
friend class RingFromColumnParser<Point>;
|
||||
friend class RingFromColumnConverter<Point>;
|
||||
ColumnPtr col{nullptr};
|
||||
};
|
||||
|
||||
|
||||
template<class Point>
|
||||
class RingFromColumnParser
|
||||
class RingFromColumnConverter
|
||||
{
|
||||
public:
|
||||
RingFromColumnParser() = default;
|
||||
|
||||
explicit RingFromColumnParser(ColumnPtr col_)
|
||||
explicit RingFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, point_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
, point_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
{
|
||||
}
|
||||
|
||||
std::vector<Ring<Point>> parse() const;
|
||||
std::vector<Ring<Point>> convert() const;
|
||||
|
||||
private:
|
||||
friend class PointFromColumnParser<Point>;
|
||||
friend class PointFromColumnConverter<Point>;
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const PointFromColumnParser<Point> point_parser{};
|
||||
const PointFromColumnConverter<Point> point_converter{};
|
||||
};
|
||||
|
||||
template<class Point>
|
||||
class PolygonFromColumnParser
|
||||
class PolygonFromColumnConverter
|
||||
{
|
||||
public:
|
||||
PolygonFromColumnParser() = default;
|
||||
PolygonFromColumnConverter() = default;
|
||||
|
||||
explicit PolygonFromColumnParser(ColumnPtr col_)
|
||||
explicit PolygonFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, ring_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
, ring_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
{
|
||||
}
|
||||
|
||||
std::vector<Polygon<Point>> parse() const;
|
||||
std::vector<Polygon<Point>> convert() const;
|
||||
|
||||
private:
|
||||
friend class MultiPolygonFromColumnParser<Point>;
|
||||
friend class MultiPolygonFromColumnConverter<Point>;
|
||||
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const RingFromColumnParser<Point> ring_parser{};
|
||||
const RingFromColumnConverter<Point> ring_converter{};
|
||||
};
|
||||
|
||||
template<class Point>
|
||||
class MultiPolygonFromColumnParser
|
||||
class MultiPolygonFromColumnConverter
|
||||
{
|
||||
public:
|
||||
MultiPolygonFromColumnParser() = default;
|
||||
MultiPolygonFromColumnConverter() = default;
|
||||
|
||||
explicit MultiPolygonFromColumnParser(ColumnPtr col_)
|
||||
explicit MultiPolygonFromColumnConverter(ColumnPtr col_)
|
||||
: col(col_)
|
||||
, polygon_parser(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
, polygon_converter(typeid_cast<const ColumnArray &>(*col_).getDataPtr())
|
||||
{}
|
||||
|
||||
std::vector<MultiPolygon<Point>> parse() const;
|
||||
std::vector<MultiPolygon<Point>> convert() const;
|
||||
|
||||
private:
|
||||
/// To prevent use-after-free and increase column lifetime.
|
||||
ColumnPtr col{nullptr};
|
||||
const PolygonFromColumnParser<Point> polygon_parser{};
|
||||
const PolygonFromColumnConverter<Point> polygon_converter{};
|
||||
};
|
||||
|
||||
|
||||
extern template class PointFromColumnParser<CartesianPoint>;
|
||||
extern template class PointFromColumnParser<GeographicPoint>;
|
||||
extern template class RingFromColumnParser<CartesianPoint>;
|
||||
extern template class RingFromColumnParser<GeographicPoint>;
|
||||
extern template class PolygonFromColumnParser<CartesianPoint>;
|
||||
extern template class PolygonFromColumnParser<GeographicPoint>;
|
||||
extern template class MultiPolygonFromColumnParser<CartesianPoint>;
|
||||
extern template class MultiPolygonFromColumnParser<GeographicPoint>;
|
||||
extern template class PointFromColumnConverter<CartesianPoint>;
|
||||
extern template class PointFromColumnConverter<GeographicPoint>;
|
||||
extern template class RingFromColumnConverter<CartesianPoint>;
|
||||
extern template class RingFromColumnConverter<GeographicPoint>;
|
||||
extern template class PolygonFromColumnConverter<CartesianPoint>;
|
||||
extern template class PolygonFromColumnConverter<GeographicPoint>;
|
||||
extern template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||
extern template class MultiPolygonFromColumnConverter<GeographicPoint>;
|
||||
|
||||
|
||||
/// To serialize Geographic or Cartesian point (a pair of numbers in both cases).
|
||||
@ -207,6 +204,7 @@ private:
|
||||
ColumnFloat64::Container & second_container;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring as Ring
|
||||
template <typename Point>
|
||||
class RingSerializer
|
||||
{
|
||||
@ -238,6 +236,7 @@ private:
|
||||
ColumnUInt64::MutablePtr offsets;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring, Polygon as Polygon
|
||||
template <typename Point>
|
||||
class PolygonSerializer
|
||||
{
|
||||
@ -277,6 +276,7 @@ private:
|
||||
ColumnUInt64::MutablePtr offsets;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring, Polygon, MultiPolygon as MultiPolygon
|
||||
template <typename Point>
|
||||
class MultiPolygonSerializer
|
||||
{
|
||||
@ -326,7 +326,7 @@ private:
|
||||
|
||||
|
||||
template <typename PType>
|
||||
struct ParserType
|
||||
struct ConverterType
|
||||
{
|
||||
using Type = PType;
|
||||
};
|
||||
@ -334,12 +334,13 @@ struct ParserType
|
||||
template <typename Point, typename F>
|
||||
static void callOnGeometryDataType(DataTypePtr type, F && f)
|
||||
{
|
||||
/// There is no Point type, because for most of geometry functions it is useless.
|
||||
if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
|
||||
return f(ParserType<RingFromColumnParser<Point>>());
|
||||
return f(ConverterType<RingFromColumnConverter<Point>>());
|
||||
if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ParserType<PolygonFromColumnParser<Point>>());
|
||||
return f(ConverterType<PolygonFromColumnConverter<Point>>());
|
||||
if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ParserType<MultiPolygonFromColumnParser<Point>>());
|
||||
return f(ConverterType<MultiPolygonFromColumnConverter<Point>>());
|
||||
throw Exception(fmt::format("Unknown geometry type {}", type->getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
@ -349,13 +350,13 @@ static void callOnTwoGeometryDataTypes(DataTypePtr left_type, DataTypePtr right_
|
||||
{
|
||||
return callOnGeometryDataType<Point>(left_type, [&](const auto & left_types)
|
||||
{
|
||||
using LeftParserType = std::decay_t<decltype(left_types)>;
|
||||
using LeftConverterType = std::decay_t<decltype(left_types)>;
|
||||
|
||||
return callOnGeometryDataType<Point>(right_type, [&](const auto & right_types)
|
||||
{
|
||||
using RightParserType = std::decay_t<decltype(right_types)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_types)>;
|
||||
|
||||
return func(LeftParserType(), RightParserType());
|
||||
return func(LeftConverterType(), RightConverterType());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
@ -59,16 +59,16 @@ public:
|
||||
|
||||
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeParser = std::decay_t<decltype(type)>;
|
||||
using Parser = typename TypeParser::Type;
|
||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = parser.parse();
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto geometries = converter.convert();
|
||||
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::area(figures[i]));
|
||||
res_data.emplace_back(boost::geometry::area(geometries[i]));
|
||||
}
|
||||
);
|
||||
|
||||
|
@ -1,89 +1,106 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnsNumber.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <string>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonConvexHull : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static const char * name;
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
// explicit FunctionPolygonConvexHull() = default;
|
||||
template <typename Point>
|
||||
class FunctionPolygonConvexHull : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonConvexHull>();
|
||||
// }
|
||||
explicit FunctionPolygonConvexHull() = default;
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonConvexHull>();
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 1;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return DataTypeCustomPolygonSerialization::nestedDataType();
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto figures = parseFigure(parser);
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return DataTypeCustomPolygonSerialization::nestedDataType();
|
||||
}
|
||||
|
||||
// PolygonSerializer<Point> serializer;
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
PolygonSerializer<Point> serializer;
|
||||
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// Polygon<Point> convex_hull{};
|
||||
// boost::geometry::convex_hull(figures[i], convex_hull);
|
||||
// serializer.add(convex_hull);
|
||||
// }
|
||||
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
// return serializer.finalize();
|
||||
// }
|
||||
if (std::is_same_v<Converter, MultiPolygonSerializer<Point>>)
|
||||
throw Exception(fmt::format("The argument of function {} could not be a MultiPolygon", getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
else
|
||||
{
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto geometries = converter.convert();
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
Polygon<Point> convex_hull{};
|
||||
boost::geometry::convex_hull(geometries[i], convex_hull);
|
||||
serializer.add(convex_hull);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonConvexHull<CartesianPoint>::name = "polygonConvexHullCartesian";
|
||||
template <>
|
||||
const char * FunctionPolygonConvexHull<CartesianPoint>::name = "polygonConvexHullCartesian";
|
||||
|
||||
|
||||
// void registerFunctionPolygonConvexHull(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonConvexHull<CartesianPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonConvexHull(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonConvexHull<CartesianPoint>>();
|
||||
}
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -1,94 +1,97 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnsNumber.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <string>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonPerimeter : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonPerimeter : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
// explicit FunctionPolygonPerimeter() = default;
|
||||
explicit FunctionPolygonPerimeter() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonPerimeter>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonPerimeter>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 1;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return std::make_shared<DataTypeFloat64>();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto figures = parseFigure(parser);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnFloat64::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
// auto res_column = ColumnFloat64::create();
|
||||
// auto & res_data = res_column->getData();
|
||||
// res_data.reserve(input_rows_count);
|
||||
callOnGeometryDataType<Point>(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 geometries = converter.convert();
|
||||
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// boost::geometry::correct(figures[i]);
|
||||
// res_data.emplace_back(boost::geometry::perimeter(figures[i]));
|
||||
// }
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::perimeter(geometries[i]));
|
||||
}
|
||||
);
|
||||
|
||||
// return res_column;
|
||||
// }
|
||||
return res_column;
|
||||
}
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonPerimeter<CartesianPoint>::name = "polygonPerimeterCartesian";
|
||||
template <>
|
||||
const char * FunctionPolygonPerimeter<CartesianPoint>::name = "polygonPerimeterCartesian";
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonPerimeter<GeographicPoint>::name = "polygonPerimeterGeographic";
|
||||
template <>
|
||||
const char * FunctionPolygonPerimeter<GeographicPoint>::name = "polygonPerimeterGeographic";
|
||||
|
||||
|
||||
// void registerFunctionPolygonPerimeter(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonPerimeter<CartesianPoint>>();
|
||||
// factory.registerFunction<FunctionPolygonPerimeter<GeographicPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonPerimeter(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonPerimeter<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonPerimeter<GeographicPoint>>();
|
||||
}
|
||||
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -1,101 +1,107 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnConst.h>
|
||||
// #include <Columns/ColumnsNumber.h>
|
||||
// #include <DataTypes/DataTypesNumber.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <utility>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonsDistance : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static inline const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonsDistance : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
// explicit FunctionPolygonsDistance() = default;
|
||||
explicit FunctionPolygonsDistance() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonsDistance>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsDistance>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 2;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return std::make_shared<DataTypeFloat64>();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnFloat64::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
// auto first = parseFigure(first_parser);
|
||||
// auto second = parseFigure(second_parser);
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
// auto res_column = ColumnFloat64::create();
|
||||
// auto & res_data = res_column->getData();
|
||||
// res_data.reserve(input_rows_count);
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// boost::geometry::correct(first[i]);
|
||||
// boost::geometry::correct(second[i]);
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
// res_data.emplace_back(boost::geometry::distance(first[i], second[i]));
|
||||
// }
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
// return res_column;
|
||||
// }
|
||||
res_data.emplace_back(boost::geometry::distance(first[i], second[i]));
|
||||
}
|
||||
});
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
return res_column;
|
||||
}
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsDistance<CartesianPoint>::name = "polygonsDistanceCartesian";
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsDistance<GeographicPoint>::name = "polygonsDistanceGeographic";
|
||||
template <>
|
||||
const char * FunctionPolygonsDistance<CartesianPoint>::name = "polygonsDistanceCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsDistance<GeographicPoint>::name = "polygonsDistanceGeographic";
|
||||
|
||||
|
||||
// void registerFunctionPolygonsDistance(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonsDistance<CartesianPoint>>();
|
||||
// factory.registerFunction<FunctionPolygonsDistance<GeographicPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonsDistance(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsDistance<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsDistance<GeographicPoint>>();
|
||||
}
|
||||
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -1,98 +1,105 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnConst.h>
|
||||
// #include <Columns/ColumnsNumber.h>
|
||||
// #include <DataTypes/DataTypesNumber.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <utility>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonsEquals : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonsEquals : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
// explicit FunctionPolygonsEquals() = default;
|
||||
explicit FunctionPolygonsEquals() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonsEquals>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsEquals>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 2;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return std::make_shared<DataTypeUInt8>();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnUInt8::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
// auto first = parseFigure(first_parser);
|
||||
// auto second = parseFigure(second_parser);
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
// auto res_column = ColumnUInt8::create();
|
||||
// auto & res_data = res_column->getData();
|
||||
// res_data.reserve(input_rows_count);
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// boost::geometry::correct(first[i]);
|
||||
// boost::geometry::correct(second[i]);
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
// /// Main work here.
|
||||
// res_data.emplace_back(boost::geometry::equals(first[i], second[i]));
|
||||
// }
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
// return res_column;
|
||||
// }
|
||||
/// Main work here.
|
||||
res_data.emplace_back(boost::geometry::equals(first[i], second[i]));
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsEquals<CartesianPoint>::name = "polygonsEqualsCartesian";
|
||||
template <>
|
||||
const char * FunctionPolygonsEquals<CartesianPoint>::name = "polygonsEqualsCartesian";
|
||||
|
||||
|
||||
// void registerFunctionPolygonsEquals(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonsEquals<CartesianPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonsEquals(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsEquals<CartesianPoint>>();
|
||||
}
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -59,15 +59,16 @@ public:
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type) {
|
||||
using LeftParserType = std::decay_t<decltype(left_type)>;
|
||||
using RightParserType = std::decay_t<decltype(right_type)>;
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
using LeftParser = typename LeftParserType::Type;
|
||||
using RightParser = typename RightParserType::Type;
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
auto first = LeftParser(arguments[0].column->convertToFullColumnIfConst()).parse();
|
||||
auto second = RightParser(arguments[1].column->convertToFullColumnIfConst()).parse();
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
|
@ -1,99 +1,105 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnConst.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <utility>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonsSymDifference : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonsSymDifference : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
// explicit FunctionPolygonsSymDifference() = default;
|
||||
explicit FunctionPolygonsSymDifference() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonsSymDifference>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsSymDifference>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 2;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
// auto first = parseFigure(first_parser);
|
||||
// auto second = parseFigure(second_parser);
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
// MultiPolygonSerializer<Point> serializer;
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
// /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// boost::geometry::correct(first[i]);
|
||||
// boost::geometry::correct(second[i]);
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
// MultiPolygon<Point> sym_difference{};
|
||||
// boost::geometry::sym_difference(first[i], second[i], sym_difference);
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
// serializer.add(sym_difference);
|
||||
// }
|
||||
MultiPolygon<Point> sym_difference{};
|
||||
boost::geometry::sym_difference(first[i], second[i], sym_difference);
|
||||
|
||||
// return serializer.finalize();
|
||||
// }
|
||||
serializer.add(sym_difference);
|
||||
}
|
||||
});
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsSymDifference<CartesianPoint>::name = "polygonsSymDifferenceCartesian";
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsSymDifference<GeographicPoint>::name = "polygonsSymDifferenceGeographic";
|
||||
template <>
|
||||
const char * FunctionPolygonsSymDifference<CartesianPoint>::name = "polygonsSymDifferenceCartesian";
|
||||
|
||||
// void registerFunctionPolygonsSymDifference(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonsSymDifference<CartesianPoint>>();
|
||||
// factory.registerFunction<FunctionPolygonsSymDifference<GeographicPoint>>();
|
||||
// }
|
||||
template <>
|
||||
const char * FunctionPolygonsSymDifference<GeographicPoint>::name = "polygonsSymDifferenceGeographic";
|
||||
|
||||
// }
|
||||
void registerFunctionPolygonsSymDifference(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsSymDifference<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsSymDifference<GeographicPoint>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,103 +1,109 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnConst.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <string>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonsUnion : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static inline const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonsUnion : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
// explicit FunctionPolygonsUnion() = default;
|
||||
explicit FunctionPolygonsUnion() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonsUnion>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsUnion>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 2;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
// auto first = parseFigure(first_parser);
|
||||
// auto second = parseFigure(second_parser);
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
// MultiPolygonSerializer<Point> serializer;
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
// /// We are not interested in some pitfalls in third-party libraries
|
||||
// /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// /// Orient the polygons correctly.
|
||||
// boost::geometry::correct(first[i]);
|
||||
// boost::geometry::correct(second[i]);
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
// MultiPolygon<Point> polygons_union{};
|
||||
// /// Main work here.
|
||||
// boost::geometry::union_(first[i], second[i], polygons_union);
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
/// Orient the polygons correctly.
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
// serializer.add(polygons_union);
|
||||
// }
|
||||
MultiPolygon<Point> polygons_union{};
|
||||
/// Main work here.
|
||||
boost::geometry::union_(first[i], second[i], polygons_union);
|
||||
|
||||
// return serializer.finalize();
|
||||
// }
|
||||
serializer.add(polygons_union);
|
||||
}
|
||||
});
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsUnion<CartesianPoint>::name = "polygonsUnionCartesian";
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsUnion<GeographicPoint>::name = "polygonsUnionGeographic";
|
||||
template <>
|
||||
const char * FunctionPolygonsUnion<CartesianPoint>::name = "polygonsUnionCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsUnion<GeographicPoint>::name = "polygonsUnionGeographic";
|
||||
|
||||
|
||||
// void registerFunctionPolygonsUnion(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonsUnion<CartesianPoint>>();
|
||||
// factory.registerFunction<FunctionPolygonsUnion<GeographicPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonsUnion(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsUnion<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsUnion<GeographicPoint>>();
|
||||
}
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -1,101 +1,108 @@
|
||||
// #include <Functions/FunctionFactory.h>
|
||||
// #include <Functions/geometryConverters.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
|
||||
// #include <boost/geometry.hpp>
|
||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
||||
// #include <boost/geometry/geometries/polygon.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
// #include <common/logger_useful.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
// #include <Columns/ColumnArray.h>
|
||||
// #include <Columns/ColumnTuple.h>
|
||||
// #include <Columns/ColumnConst.h>
|
||||
// #include <Columns/ColumnsNumber.h>
|
||||
// #include <DataTypes/DataTypesNumber.h>
|
||||
// #include <DataTypes/DataTypeArray.h>
|
||||
// #include <DataTypes/DataTypeTuple.h>
|
||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
// #include <memory>
|
||||
// #include <utility>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
// namespace DB
|
||||
// {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// template <typename Point>
|
||||
// class FunctionPolygonsWithin : public IFunction
|
||||
// {
|
||||
// public:
|
||||
// static inline const char * name;
|
||||
template <typename Point>
|
||||
class FunctionPolygonsWithin : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
// explicit FunctionPolygonsWithin() = default;
|
||||
explicit FunctionPolygonsWithin() = default;
|
||||
|
||||
// static FunctionPtr create(const Context &)
|
||||
// {
|
||||
// return std::make_shared<FunctionPolygonsWithin>();
|
||||
// }
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsWithin>();
|
||||
}
|
||||
|
||||
// String getName() const override
|
||||
// {
|
||||
// return name;
|
||||
// }
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
// bool isVariadic() const override
|
||||
// {
|
||||
// return false;
|
||||
// }
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
// size_t getNumberOfArguments() const override
|
||||
// {
|
||||
// return 2;
|
||||
// }
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
// {
|
||||
// return std::make_shared<DataTypeUInt8>();
|
||||
// }
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
// {
|
||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnUInt8::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
// auto first = parseFigure(first_parser);
|
||||
// auto second = parseFigure(second_parser);
|
||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||
|
||||
// auto res_column = ColumnUInt8::create();
|
||||
// auto & res_data = res_column->getData();
|
||||
using LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
// /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
// for (size_t i = 0; i < input_rows_count; i++)
|
||||
// {
|
||||
// boost::geometry::correct(first[i]);
|
||||
// boost::geometry::correct(second[i]);
|
||||
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||
|
||||
// res_data.emplace_back(boost::geometry::within(first[i], second[i]));
|
||||
// }
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
// return res_column;
|
||||
// }
|
||||
res_data.emplace_back(boost::geometry::within(first[i], second[i]));
|
||||
}
|
||||
});
|
||||
|
||||
// bool useDefaultImplementationForConstants() const override
|
||||
// {
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsWithin<CartesianPoint>::name = "polygonsWithinCartesian";
|
||||
template <>
|
||||
const char * FunctionPolygonsWithin<CartesianPoint>::name = "polygonsWithinCartesian";
|
||||
|
||||
// template <>
|
||||
// const char * FunctionPolygonsWithin<GeographicPoint>::name = "polygonsWithinGeographic";
|
||||
template <>
|
||||
const char * FunctionPolygonsWithin<GeographicPoint>::name = "polygonsWithinGeographic";
|
||||
|
||||
|
||||
// void registerFunctionPolygonsWithin(FunctionFactory & factory)
|
||||
// {
|
||||
// factory.registerFunction<FunctionPolygonsWithin<CartesianPoint>>();
|
||||
// factory.registerFunction<FunctionPolygonsWithin<GeographicPoint>>();
|
||||
// }
|
||||
void registerFunctionPolygonsWithin(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsWithin<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsWithin<GeographicPoint>>();
|
||||
}
|
||||
|
||||
// }
|
||||
}
|
||||
|
@ -11,20 +11,19 @@ void registerFunctionGeoDistance(FunctionFactory & factory);
|
||||
void registerFunctionPointInEllipses(FunctionFactory & factory);
|
||||
void registerFunctionPointInPolygon(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsIntersection(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonsUnion(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsUnion(FunctionFactory & factory);
|
||||
void registerFunctionPolygonArea(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonConvexHull(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonsSymDifference(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonsEquals(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonsDistance(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonsWithin(FunctionFactory & factory);
|
||||
// void registerFunctionPolygonPerimeter(FunctionFactory & factory);
|
||||
void registerFunctionPolygonConvexHull(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsSymDifference(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsEquals(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsDistance(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsWithin(FunctionFactory & factory);
|
||||
void registerFunctionPolygonPerimeter(FunctionFactory & factory);
|
||||
void registerFunctionGeohashEncode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashDecode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
||||
void registerFunctionWkt(FunctionFactory & factory);
|
||||
void registerFunctionReadWkt(FunctionFactory & factory);
|
||||
void registerFunctionDescribeGeometry(FunctionFactory & factory);
|
||||
void registerFunctionSvg(FunctionFactory & factory);
|
||||
|
||||
#if USE_H3
|
||||
@ -50,20 +49,19 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
||||
registerFunctionPointInEllipses(factory);
|
||||
registerFunctionPointInPolygon(factory);
|
||||
registerFunctionPolygonsIntersection(factory);
|
||||
// registerFunctionPolygonsUnion(factory);
|
||||
registerFunctionPolygonsUnion(factory);
|
||||
registerFunctionPolygonArea(factory);
|
||||
// registerFunctionPolygonConvexHull(factory);
|
||||
// registerFunctionPolygonsSymDifference(factory);
|
||||
// registerFunctionPolygonsEquals(factory);
|
||||
// registerFunctionPolygonsDistance(factory);
|
||||
// registerFunctionPolygonsWithin(factory);
|
||||
// registerFunctionPolygonPerimeter(factory);
|
||||
registerFunctionPolygonConvexHull(factory);
|
||||
registerFunctionPolygonsSymDifference(factory);
|
||||
registerFunctionPolygonsEquals(factory);
|
||||
registerFunctionPolygonsDistance(factory);
|
||||
registerFunctionPolygonsWithin(factory);
|
||||
registerFunctionPolygonPerimeter(factory);
|
||||
registerFunctionGeohashEncode(factory);
|
||||
registerFunctionGeohashDecode(factory);
|
||||
registerFunctionGeohashesInBox(factory);
|
||||
registerFunctionWkt(factory);
|
||||
registerFunctionReadWkt(factory);
|
||||
registerFunctionDescribeGeometry(factory);
|
||||
registerFunctionSvg(factory);
|
||||
|
||||
#if USE_H3
|
||||
|
@ -69,10 +69,10 @@ public:
|
||||
|
||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeParser = std::decay_t<decltype(type)>;
|
||||
using Parser = typename TypeParser::Type;
|
||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = parser.parse();
|
||||
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;
|
||||
@ -89,7 +89,7 @@ public:
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
|
@ -42,10 +42,10 @@ public:
|
||||
|
||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeParser = std::decay_t<decltype(type)>;
|
||||
using Parser = typename TypeParser::Type;
|
||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = parser.parse();
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto figures = converter.convert();
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
@ -56,7 +56,7 @@ public:
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
|
@ -1 +1 @@
|
||||
select polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]);
|
||||
select polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (0., 0.)]]]);
|
Loading…
Reference in New Issue
Block a user