mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +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>
|
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 = typeid_cast<const ColumnTuple *>(col.get());
|
||||||
const auto & tuple_columns = tuple->getColumns();
|
const auto & tuple_columns = tuple->getColumns();
|
||||||
@ -44,7 +44,7 @@ std::vector<Point> PointFromColumnParser<Point>::parseImpl(size_t shift, size_t
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <typename Point>
|
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();
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||||
size_t prev_offset = 0;
|
size_t prev_offset = 0;
|
||||||
@ -52,7 +52,7 @@ std::vector<Ring<Point>> RingFromColumnParser<Point>::parse() const
|
|||||||
answer.reserve(offsets.size());
|
answer.reserve(offsets.size());
|
||||||
for (size_t offset : offsets)
|
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());
|
answer.emplace_back(tmp.begin(), tmp.end());
|
||||||
prev_offset = offset;
|
prev_offset = offset;
|
||||||
}
|
}
|
||||||
@ -60,11 +60,11 @@ std::vector<Ring<Point>> RingFromColumnParser<Point>::parse() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <typename Point>
|
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();
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||||
std::vector<Polygon<Point>> answer(offsets.size());
|
std::vector<Polygon<Point>> answer(offsets.size());
|
||||||
auto all_rings = ring_parser.parse();
|
auto all_rings = ring_converter.convert();
|
||||||
|
|
||||||
auto prev_offset = 0;
|
auto prev_offset = 0;
|
||||||
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
||||||
@ -82,13 +82,13 @@ std::vector<Polygon<Point>> PolygonFromColumnParser<Point>::parse() const
|
|||||||
|
|
||||||
|
|
||||||
template <typename Point>
|
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();
|
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||||
size_t prev_offset = 0;
|
size_t prev_offset = 0;
|
||||||
std::vector<MultiPolygon<Point>> answer(offsets.size());
|
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 iter = 0; iter < offsets.size(); ++iter)
|
||||||
{
|
{
|
||||||
@ -101,14 +101,14 @@ std::vector<MultiPolygon<Point>> MultiPolygonFromColumnParser<Point>::parse() co
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
template class PointFromColumnParser<CartesianPoint>;
|
template class PointFromColumnConverter<CartesianPoint>;
|
||||||
template class PointFromColumnParser<GeographicPoint>;
|
template class PointFromColumnConverter<GeographicPoint>;
|
||||||
template class RingFromColumnParser<CartesianPoint>;
|
template class RingFromColumnConverter<CartesianPoint>;
|
||||||
template class RingFromColumnParser<GeographicPoint>;
|
template class RingFromColumnConverter<GeographicPoint>;
|
||||||
template class PolygonFromColumnParser<CartesianPoint>;
|
template class PolygonFromColumnConverter<CartesianPoint>;
|
||||||
template class PolygonFromColumnParser<GeographicPoint>;
|
template class PolygonFromColumnConverter<GeographicPoint>;
|
||||||
template class MultiPolygonFromColumnParser<CartesianPoint>;
|
template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||||
template class MultiPolygonFromColumnParser<GeographicPoint>;
|
template class MultiPolygonFromColumnConverter<GeographicPoint>;
|
||||||
|
|
||||||
template <typename Point, template<typename> typename Desired>
|
template <typename Point, template<typename> typename Desired>
|
||||||
void checkColumnTypeOrThrow(const ColumnWithTypeAndName & column)
|
void checkColumnTypeOrThrow(const ColumnWithTypeAndName & column)
|
||||||
|
@ -60,109 +60,106 @@ using GeographicGeometry = Geometry<GeographicPoint>;
|
|||||||
|
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class RingFromColumnParser;
|
class RingFromColumnConverter;
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class PolygonFromColumnParser;
|
class PolygonFromColumnConverter;
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class MultiPolygonFromColumnParser;
|
class MultiPolygonFromColumnConverter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class which takes some boost type and returns a pair of numbers.
|
* 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.
|
* They are (x,y) in case of cartesian coordinated and (lon,lat) in case of geographic.
|
||||||
*/
|
*/
|
||||||
template <typename Point>
|
template <typename Point>
|
||||||
class PointFromColumnParser
|
class PointFromColumnConverter
|
||||||
{
|
{
|
||||||
public:
|
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:
|
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};
|
ColumnPtr col{nullptr};
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class RingFromColumnParser
|
class RingFromColumnConverter
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
RingFromColumnParser() = default;
|
explicit RingFromColumnConverter(ColumnPtr col_)
|
||||||
|
|
||||||
explicit RingFromColumnParser(ColumnPtr col_)
|
|
||||||
: col(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:
|
private:
|
||||||
friend class PointFromColumnParser<Point>;
|
friend class PointFromColumnConverter<Point>;
|
||||||
/// To prevent use-after-free and increase column lifetime.
|
/// To prevent use-after-free and increase column lifetime.
|
||||||
ColumnPtr col{nullptr};
|
ColumnPtr col{nullptr};
|
||||||
const PointFromColumnParser<Point> point_parser{};
|
const PointFromColumnConverter<Point> point_converter{};
|
||||||
};
|
};
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class PolygonFromColumnParser
|
class PolygonFromColumnConverter
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PolygonFromColumnParser() = default;
|
PolygonFromColumnConverter() = default;
|
||||||
|
|
||||||
explicit PolygonFromColumnParser(ColumnPtr col_)
|
explicit PolygonFromColumnConverter(ColumnPtr col_)
|
||||||
: col(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:
|
private:
|
||||||
friend class MultiPolygonFromColumnParser<Point>;
|
friend class MultiPolygonFromColumnConverter<Point>;
|
||||||
|
|
||||||
/// To prevent use-after-free and increase column lifetime.
|
/// To prevent use-after-free and increase column lifetime.
|
||||||
ColumnPtr col{nullptr};
|
ColumnPtr col{nullptr};
|
||||||
const RingFromColumnParser<Point> ring_parser{};
|
const RingFromColumnConverter<Point> ring_converter{};
|
||||||
};
|
};
|
||||||
|
|
||||||
template<class Point>
|
template<class Point>
|
||||||
class MultiPolygonFromColumnParser
|
class MultiPolygonFromColumnConverter
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
MultiPolygonFromColumnParser() = default;
|
MultiPolygonFromColumnConverter() = default;
|
||||||
|
|
||||||
explicit MultiPolygonFromColumnParser(ColumnPtr col_)
|
explicit MultiPolygonFromColumnConverter(ColumnPtr col_)
|
||||||
: col(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:
|
private:
|
||||||
/// To prevent use-after-free and increase column lifetime.
|
/// To prevent use-after-free and increase column lifetime.
|
||||||
ColumnPtr col{nullptr};
|
ColumnPtr col{nullptr};
|
||||||
const PolygonFromColumnParser<Point> polygon_parser{};
|
const PolygonFromColumnConverter<Point> polygon_converter{};
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
extern template class PointFromColumnParser<CartesianPoint>;
|
extern template class PointFromColumnConverter<CartesianPoint>;
|
||||||
extern template class PointFromColumnParser<GeographicPoint>;
|
extern template class PointFromColumnConverter<GeographicPoint>;
|
||||||
extern template class RingFromColumnParser<CartesianPoint>;
|
extern template class RingFromColumnConverter<CartesianPoint>;
|
||||||
extern template class RingFromColumnParser<GeographicPoint>;
|
extern template class RingFromColumnConverter<GeographicPoint>;
|
||||||
extern template class PolygonFromColumnParser<CartesianPoint>;
|
extern template class PolygonFromColumnConverter<CartesianPoint>;
|
||||||
extern template class PolygonFromColumnParser<GeographicPoint>;
|
extern template class PolygonFromColumnConverter<GeographicPoint>;
|
||||||
extern template class MultiPolygonFromColumnParser<CartesianPoint>;
|
extern template class MultiPolygonFromColumnConverter<CartesianPoint>;
|
||||||
extern template class MultiPolygonFromColumnParser<GeographicPoint>;
|
extern template class MultiPolygonFromColumnConverter<GeographicPoint>;
|
||||||
|
|
||||||
|
|
||||||
/// To serialize Geographic or Cartesian point (a pair of numbers in both cases).
|
/// To serialize Geographic or Cartesian point (a pair of numbers in both cases).
|
||||||
@ -207,6 +204,7 @@ private:
|
|||||||
ColumnFloat64::Container & second_container;
|
ColumnFloat64::Container & second_container;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Serialize Point, Ring as Ring
|
||||||
template <typename Point>
|
template <typename Point>
|
||||||
class RingSerializer
|
class RingSerializer
|
||||||
{
|
{
|
||||||
@ -238,6 +236,7 @@ private:
|
|||||||
ColumnUInt64::MutablePtr offsets;
|
ColumnUInt64::MutablePtr offsets;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Serialize Point, Ring, Polygon as Polygon
|
||||||
template <typename Point>
|
template <typename Point>
|
||||||
class PolygonSerializer
|
class PolygonSerializer
|
||||||
{
|
{
|
||||||
@ -277,6 +276,7 @@ private:
|
|||||||
ColumnUInt64::MutablePtr offsets;
|
ColumnUInt64::MutablePtr offsets;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Serialize Point, Ring, Polygon, MultiPolygon as MultiPolygon
|
||||||
template <typename Point>
|
template <typename Point>
|
||||||
class MultiPolygonSerializer
|
class MultiPolygonSerializer
|
||||||
{
|
{
|
||||||
@ -326,7 +326,7 @@ private:
|
|||||||
|
|
||||||
|
|
||||||
template <typename PType>
|
template <typename PType>
|
||||||
struct ParserType
|
struct ConverterType
|
||||||
{
|
{
|
||||||
using Type = PType;
|
using Type = PType;
|
||||||
};
|
};
|
||||||
@ -334,12 +334,13 @@ struct ParserType
|
|||||||
template <typename Point, typename F>
|
template <typename Point, typename F>
|
||||||
static void callOnGeometryDataType(DataTypePtr type, F && 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))
|
if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
|
||||||
return f(ParserType<RingFromColumnParser<Point>>());
|
return f(ConverterType<RingFromColumnConverter<Point>>());
|
||||||
if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
||||||
return f(ParserType<PolygonFromColumnParser<Point>>());
|
return f(ConverterType<PolygonFromColumnConverter<Point>>());
|
||||||
if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
|
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);
|
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)
|
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)
|
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)
|
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||||
{
|
{
|
||||||
using TypeParser = std::decay_t<decltype(type)>;
|
using TypeConverter = std::decay_t<decltype(type)>;
|
||||||
using Parser = typename TypeParser::Type;
|
using Converter = typename TypeConverter::Type;
|
||||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||||
auto figures = parser.parse();
|
auto geometries = converter.convert();
|
||||||
|
|
||||||
auto & res_data = res_column->getData();
|
auto & res_data = res_column->getData();
|
||||||
res_data.reserve(input_rows_count);
|
res_data.reserve(input_rows_count);
|
||||||
|
|
||||||
for (size_t i = 0; i < input_rows_count; i++)
|
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/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <string>
|
#include <string>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
namespace ErrorCodes
|
||||||
// class FunctionPolygonConvexHull : public IFunction
|
{
|
||||||
// {
|
extern const int BAD_ARGUMENTS;
|
||||||
// public:
|
}
|
||||||
// static const char * name;
|
|
||||||
|
|
||||||
// explicit FunctionPolygonConvexHull() = default;
|
template <typename Point>
|
||||||
|
class FunctionPolygonConvexHull : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static const char * name;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
explicit FunctionPolygonConvexHull() = default;
|
||||||
// {
|
|
||||||
// return std::make_shared<FunctionPolygonConvexHull>();
|
|
||||||
// }
|
|
||||||
|
|
||||||
// String getName() const override
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return name;
|
return std::make_shared<FunctionPolygonConvexHull>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return 1;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return DataTypeCustomPolygonSerialization::nestedDataType();
|
return 1;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// auto parser = getConverterBasedOnType<Point>(arguments[0]);
|
return DataTypeCustomPolygonSerialization::nestedDataType();
|
||||||
// auto figures = parseFigure(parser);
|
}
|
||||||
|
|
||||||
// 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++)
|
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||||
// {
|
{
|
||||||
// Polygon<Point> convex_hull{};
|
using TypeConverter = std::decay_t<decltype(type)>;
|
||||||
// boost::geometry::convex_hull(figures[i], convex_hull);
|
using Converter = typename TypeConverter::Type;
|
||||||
// serializer.add(convex_hull);
|
|
||||||
// }
|
|
||||||
|
|
||||||
// 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
|
for (size_t i = 0; i < input_rows_count; i++)
|
||||||
// {
|
{
|
||||||
// return true;
|
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 <>
|
template <>
|
||||||
// const char * FunctionPolygonConvexHull<CartesianPoint>::name = "polygonConvexHullCartesian";
|
const char * FunctionPolygonConvexHull<CartesianPoint>::name = "polygonConvexHullCartesian";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonConvexHull(FunctionFactory & factory)
|
void registerFunctionPolygonConvexHull(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonConvexHull<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonConvexHull<CartesianPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -1,94 +1,97 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <string>
|
#include <string>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonPerimeter : public IFunction
|
class FunctionPolygonPerimeter : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static const char * name;
|
static const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonPerimeter() = default;
|
explicit FunctionPolygonPerimeter() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonPerimeter>();
|
return std::make_shared<FunctionPolygonPerimeter>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 1;
|
return 1;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return std::make_shared<DataTypeFloat64>();
|
return std::make_shared<DataTypeFloat64>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto parser = getConverterBasedOnType<Point>(arguments[0]);
|
auto res_column = ColumnFloat64::create();
|
||||||
// auto figures = parseFigure(parser);
|
auto & res_data = res_column->getData();
|
||||||
|
res_data.reserve(input_rows_count);
|
||||||
|
|
||||||
// auto res_column = ColumnFloat64::create();
|
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||||
// auto & res_data = res_column->getData();
|
{
|
||||||
// res_data.reserve(input_rows_count);
|
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++)
|
for (size_t i = 0; i < input_rows_count; i++)
|
||||||
// {
|
res_data.emplace_back(boost::geometry::perimeter(geometries[i]));
|
||||||
// boost::geometry::correct(figures[i]);
|
}
|
||||||
// res_data.emplace_back(boost::geometry::perimeter(figures[i]));
|
);
|
||||||
// }
|
|
||||||
|
|
||||||
// return res_column;
|
return res_column;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool useDefaultImplementationForConstants() const override
|
bool useDefaultImplementationForConstants() const override
|
||||||
// {
|
{
|
||||||
// return true;
|
return true;
|
||||||
// }
|
}
|
||||||
// };
|
};
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonPerimeter<CartesianPoint>::name = "polygonPerimeterCartesian";
|
const char * FunctionPolygonPerimeter<CartesianPoint>::name = "polygonPerimeterCartesian";
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonPerimeter<GeographicPoint>::name = "polygonPerimeterGeographic";
|
const char * FunctionPolygonPerimeter<GeographicPoint>::name = "polygonPerimeterGeographic";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonPerimeter(FunctionFactory & factory)
|
void registerFunctionPolygonPerimeter(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonPerimeter<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonPerimeter<CartesianPoint>>();
|
||||||
// factory.registerFunction<FunctionPolygonPerimeter<GeographicPoint>>();
|
factory.registerFunction<FunctionPolygonPerimeter<GeographicPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -1,101 +1,107 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
// #include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
// #include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <utility>
|
#include <utility>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonsDistance : public IFunction
|
class FunctionPolygonsDistance : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static inline const char * name;
|
static inline const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonsDistance() = default;
|
explicit FunctionPolygonsDistance() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonsDistance>();
|
return std::make_shared<FunctionPolygonsDistance>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 2;
|
return 2;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return std::make_shared<DataTypeFloat64>();
|
return std::make_shared<DataTypeFloat64>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
auto res_column = ColumnFloat64::create();
|
||||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
auto & res_data = res_column->getData();
|
||||||
|
res_data.reserve(input_rows_count);
|
||||||
|
|
||||||
// auto first = parseFigure(first_parser);
|
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||||
// auto second = parseFigure(second_parser);
|
{
|
||||||
|
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||||
|
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||||
|
|
||||||
// auto res_column = ColumnFloat64::create();
|
using LeftConverter = typename LeftConverterType::Type;
|
||||||
// auto & res_data = res_column->getData();
|
using RightConverter = typename RightConverterType::Type;
|
||||||
// res_data.reserve(input_rows_count);
|
|
||||||
|
|
||||||
// for (size_t i = 0; i < input_rows_count; i++)
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
// {
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
// boost::geometry::correct(first[i]);
|
|
||||||
// boost::geometry::correct(second[i]);
|
|
||||||
|
|
||||||
// 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 res_column;
|
||||||
// {
|
}
|
||||||
// return true;
|
|
||||||
// }
|
|
||||||
// };
|
|
||||||
|
|
||||||
// template <>
|
bool useDefaultImplementationForConstants() const override
|
||||||
// const char * FunctionPolygonsDistance<CartesianPoint>::name = "polygonsDistanceCartesian";
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsDistance<GeographicPoint>::name = "polygonsDistanceGeographic";
|
const char * FunctionPolygonsDistance<CartesianPoint>::name = "polygonsDistanceCartesian";
|
||||||
|
|
||||||
|
template <>
|
||||||
|
const char * FunctionPolygonsDistance<GeographicPoint>::name = "polygonsDistanceGeographic";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonsDistance(FunctionFactory & factory)
|
void registerFunctionPolygonsDistance(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonsDistance<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonsDistance<CartesianPoint>>();
|
||||||
// factory.registerFunction<FunctionPolygonsDistance<GeographicPoint>>();
|
factory.registerFunction<FunctionPolygonsDistance<GeographicPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -1,98 +1,105 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
// #include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
// #include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <utility>
|
#include <utility>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonsEquals : public IFunction
|
class FunctionPolygonsEquals : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static const char * name;
|
static const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonsEquals() = default;
|
explicit FunctionPolygonsEquals() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonsEquals>();
|
return std::make_shared<FunctionPolygonsEquals>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 2;
|
return 2;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
auto res_column = ColumnUInt8::create();
|
||||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
auto & res_data = res_column->getData();
|
||||||
|
res_data.reserve(input_rows_count);
|
||||||
|
|
||||||
// auto first = parseFigure(first_parser);
|
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||||
// auto second = parseFigure(second_parser);
|
{
|
||||||
|
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||||
|
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||||
|
|
||||||
// auto res_column = ColumnUInt8::create();
|
using LeftConverter = typename LeftConverterType::Type;
|
||||||
// auto & res_data = res_column->getData();
|
using RightConverter = typename RightConverterType::Type;
|
||||||
// res_data.reserve(input_rows_count);
|
|
||||||
|
|
||||||
// for (size_t i = 0; i < input_rows_count; i++)
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
// {
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
// boost::geometry::correct(first[i]);
|
|
||||||
// boost::geometry::correct(second[i]);
|
|
||||||
|
|
||||||
// /// Main work here.
|
for (size_t i = 0; i < input_rows_count; i++)
|
||||||
// res_data.emplace_back(boost::geometry::equals(first[i], second[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 res_column;
|
||||||
// {
|
}
|
||||||
// return true;
|
|
||||||
// }
|
bool useDefaultImplementationForConstants() const override
|
||||||
// };
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsEquals<CartesianPoint>::name = "polygonsEqualsCartesian";
|
const char * FunctionPolygonsEquals<CartesianPoint>::name = "polygonsEqualsCartesian";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonsEquals(FunctionFactory & factory)
|
void registerFunctionPolygonsEquals(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonsEquals<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonsEquals<CartesianPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -59,15 +59,16 @@ public:
|
|||||||
{
|
{
|
||||||
MultiPolygonSerializer<Point> serializer;
|
MultiPolygonSerializer<Point> serializer;
|
||||||
|
|
||||||
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type) {
|
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)>;
|
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||||
|
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||||
|
|
||||||
using LeftParser = typename LeftParserType::Type;
|
using LeftConverter = typename LeftConverterType::Type;
|
||||||
using RightParser = typename RightParserType::Type;
|
using RightConverter = typename RightConverterType::Type;
|
||||||
|
|
||||||
auto first = LeftParser(arguments[0].column->convertToFullColumnIfConst()).parse();
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
auto second = RightParser(arguments[1].column->convertToFullColumnIfConst()).parse();
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
|
|
||||||
/// We are not interested in some pitfalls in third-party libraries
|
/// We are not interested in some pitfalls in third-party libraries
|
||||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||||
|
@ -1,99 +1,105 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <utility>
|
#include <utility>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonsSymDifference : public IFunction
|
class FunctionPolygonsSymDifference : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static const char * name;
|
static const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonsSymDifference() = default;
|
explicit FunctionPolygonsSymDifference() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonsSymDifference>();
|
return std::make_shared<FunctionPolygonsSymDifference>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 2;
|
return 2;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
MultiPolygonSerializer<Point> serializer;
|
||||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
|
||||||
|
|
||||||
// auto first = parseFigure(first_parser);
|
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||||
// auto second = parseFigure(second_parser);
|
{
|
||||||
|
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)
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
// for (size_t i = 0; i < input_rows_count; i++)
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
// {
|
|
||||||
// boost::geometry::correct(first[i]);
|
|
||||||
// boost::geometry::correct(second[i]);
|
|
||||||
|
|
||||||
// MultiPolygon<Point> sym_difference{};
|
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||||
// boost::geometry::sym_difference(first[i], second[i], sym_difference);
|
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 serializer.finalize();
|
||||||
// {
|
}
|
||||||
// return true;
|
|
||||||
// }
|
|
||||||
// };
|
|
||||||
|
|
||||||
// template <>
|
bool useDefaultImplementationForConstants() const override
|
||||||
// const char * FunctionPolygonsSymDifference<CartesianPoint>::name = "polygonsSymDifferenceCartesian";
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsSymDifference<GeographicPoint>::name = "polygonsSymDifferenceGeographic";
|
const char * FunctionPolygonsSymDifference<CartesianPoint>::name = "polygonsSymDifferenceCartesian";
|
||||||
|
|
||||||
// void registerFunctionPolygonsSymDifference(FunctionFactory & factory)
|
template <>
|
||||||
// {
|
const char * FunctionPolygonsSymDifference<GeographicPoint>::name = "polygonsSymDifferenceGeographic";
|
||||||
// factory.registerFunction<FunctionPolygonsSymDifference<CartesianPoint>>();
|
|
||||||
// factory.registerFunction<FunctionPolygonsSymDifference<GeographicPoint>>();
|
|
||||||
// }
|
|
||||||
|
|
||||||
// }
|
void registerFunctionPolygonsSymDifference(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionPolygonsSymDifference<CartesianPoint>>();
|
||||||
|
factory.registerFunction<FunctionPolygonsSymDifference<GeographicPoint>>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
@ -1,103 +1,109 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <string>
|
#include <string>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonsUnion : public IFunction
|
class FunctionPolygonsUnion : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static inline const char * name;
|
static inline const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonsUnion() = default;
|
explicit FunctionPolygonsUnion() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonsUnion>();
|
return std::make_shared<FunctionPolygonsUnion>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 2;
|
return 2;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
MultiPolygonSerializer<Point> serializer;
|
||||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
|
||||||
|
|
||||||
// auto first = parseFigure(first_parser);
|
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||||
// auto second = parseFigure(second_parser);
|
{
|
||||||
|
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
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
// /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
// for (size_t i = 0; i < input_rows_count; i++)
|
|
||||||
// {
|
|
||||||
// /// Orient the polygons correctly.
|
|
||||||
// boost::geometry::correct(first[i]);
|
|
||||||
// boost::geometry::correct(second[i]);
|
|
||||||
|
|
||||||
// MultiPolygon<Point> polygons_union{};
|
/// We are not interested in some pitfalls in third-party libraries
|
||||||
// /// Main work here.
|
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||||
// boost::geometry::union_(first[i], second[i], polygons_union);
|
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 serializer.finalize();
|
||||||
// {
|
}
|
||||||
// return true;
|
|
||||||
// }
|
|
||||||
// };
|
|
||||||
|
|
||||||
// template <>
|
bool useDefaultImplementationForConstants() const override
|
||||||
// const char * FunctionPolygonsUnion<CartesianPoint>::name = "polygonsUnionCartesian";
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsUnion<GeographicPoint>::name = "polygonsUnionGeographic";
|
const char * FunctionPolygonsUnion<CartesianPoint>::name = "polygonsUnionCartesian";
|
||||||
|
|
||||||
|
template <>
|
||||||
|
const char * FunctionPolygonsUnion<GeographicPoint>::name = "polygonsUnionGeographic";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonsUnion(FunctionFactory & factory)
|
void registerFunctionPolygonsUnion(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonsUnion<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonsUnion<CartesianPoint>>();
|
||||||
// factory.registerFunction<FunctionPolygonsUnion<GeographicPoint>>();
|
factory.registerFunction<FunctionPolygonsUnion<GeographicPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -1,101 +1,108 @@
|
|||||||
// #include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
// #include <Functions/geometryConverters.h>
|
#include <Functions/geometryConverters.h>
|
||||||
|
|
||||||
// #include <boost/geometry.hpp>
|
#include <boost/geometry.hpp>
|
||||||
// #include <boost/geometry/geometries/point_xy.hpp>
|
#include <boost/geometry/geometries/point_xy.hpp>
|
||||||
// #include <boost/geometry/geometries/polygon.hpp>
|
#include <boost/geometry/geometries/polygon.hpp>
|
||||||
|
|
||||||
// #include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
// #include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
// #include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
// #include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
// #include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
// #include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
// #include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
// #include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
// #include <DataTypes/DataTypeCustomGeo.h>
|
#include <DataTypes/DataTypeCustomGeo.h>
|
||||||
|
|
||||||
// #include <memory>
|
#include <memory>
|
||||||
// #include <utility>
|
#include <utility>
|
||||||
|
|
||||||
// namespace DB
|
namespace DB
|
||||||
// {
|
{
|
||||||
|
|
||||||
// template <typename Point>
|
template <typename Point>
|
||||||
// class FunctionPolygonsWithin : public IFunction
|
class FunctionPolygonsWithin : public IFunction
|
||||||
// {
|
{
|
||||||
// public:
|
public:
|
||||||
// static inline const char * name;
|
static inline const char * name;
|
||||||
|
|
||||||
// explicit FunctionPolygonsWithin() = default;
|
explicit FunctionPolygonsWithin() = default;
|
||||||
|
|
||||||
// static FunctionPtr create(const Context &)
|
static FunctionPtr create(const Context &)
|
||||||
// {
|
{
|
||||||
// return std::make_shared<FunctionPolygonsWithin>();
|
return std::make_shared<FunctionPolygonsWithin>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// String getName() const override
|
String getName() const override
|
||||||
// {
|
{
|
||||||
// return name;
|
return name;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// bool isVariadic() const override
|
bool isVariadic() const override
|
||||||
// {
|
{
|
||||||
// return false;
|
return false;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// size_t getNumberOfArguments() const override
|
size_t getNumberOfArguments() const override
|
||||||
// {
|
{
|
||||||
// return 2;
|
return 2;
|
||||||
// }
|
}
|
||||||
|
|
||||||
// DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
// {
|
{
|
||||||
// return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||||
// {
|
{
|
||||||
// auto first_parser = getConverterBasedOnType<Point>(arguments[0]);
|
auto res_column = ColumnUInt8::create();
|
||||||
// auto second_parser = getConverterBasedOnType<Point>(arguments[1]);
|
auto & res_data = res_column->getData();
|
||||||
|
res_data.reserve(input_rows_count);
|
||||||
|
|
||||||
// auto first = parseFigure(first_parser);
|
callOnTwoGeometryDataTypes<Point>(arguments[0].type, arguments[1].type, [&](const auto & left_type, const auto & right_type)
|
||||||
// auto second = parseFigure(second_parser);
|
{
|
||||||
|
using LeftConverterType = std::decay_t<decltype(left_type)>;
|
||||||
|
using RightConverterType = std::decay_t<decltype(right_type)>;
|
||||||
|
|
||||||
// auto res_column = ColumnUInt8::create();
|
using LeftConverter = typename LeftConverterType::Type;
|
||||||
// auto & res_data = res_column->getData();
|
using RightConverter = typename RightConverterType::Type;
|
||||||
|
|
||||||
// /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
auto first = LeftConverter(arguments[0].column->convertToFullColumnIfConst()).convert();
|
||||||
// for (size_t i = 0; i < input_rows_count; i++)
|
auto second = RightConverter(arguments[1].column->convertToFullColumnIfConst()).convert();
|
||||||
// {
|
|
||||||
// boost::geometry::correct(first[i]);
|
|
||||||
// boost::geometry::correct(second[i]);
|
|
||||||
|
|
||||||
// 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 res_column;
|
||||||
// {
|
}
|
||||||
// return true;
|
|
||||||
// }
|
bool useDefaultImplementationForConstants() const override
|
||||||
// };
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsWithin<CartesianPoint>::name = "polygonsWithinCartesian";
|
const char * FunctionPolygonsWithin<CartesianPoint>::name = "polygonsWithinCartesian";
|
||||||
|
|
||||||
// template <>
|
template <>
|
||||||
// const char * FunctionPolygonsWithin<GeographicPoint>::name = "polygonsWithinGeographic";
|
const char * FunctionPolygonsWithin<GeographicPoint>::name = "polygonsWithinGeographic";
|
||||||
|
|
||||||
|
|
||||||
// void registerFunctionPolygonsWithin(FunctionFactory & factory)
|
void registerFunctionPolygonsWithin(FunctionFactory & factory)
|
||||||
// {
|
{
|
||||||
// factory.registerFunction<FunctionPolygonsWithin<CartesianPoint>>();
|
factory.registerFunction<FunctionPolygonsWithin<CartesianPoint>>();
|
||||||
// factory.registerFunction<FunctionPolygonsWithin<GeographicPoint>>();
|
factory.registerFunction<FunctionPolygonsWithin<GeographicPoint>>();
|
||||||
// }
|
}
|
||||||
|
|
||||||
// }
|
}
|
||||||
|
@ -11,20 +11,19 @@ void registerFunctionGeoDistance(FunctionFactory & factory);
|
|||||||
void registerFunctionPointInEllipses(FunctionFactory & factory);
|
void registerFunctionPointInEllipses(FunctionFactory & factory);
|
||||||
void registerFunctionPointInPolygon(FunctionFactory & factory);
|
void registerFunctionPointInPolygon(FunctionFactory & factory);
|
||||||
void registerFunctionPolygonsIntersection(FunctionFactory & factory);
|
void registerFunctionPolygonsIntersection(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonsUnion(FunctionFactory & factory);
|
void registerFunctionPolygonsUnion(FunctionFactory & factory);
|
||||||
void registerFunctionPolygonArea(FunctionFactory & factory);
|
void registerFunctionPolygonArea(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonConvexHull(FunctionFactory & factory);
|
void registerFunctionPolygonConvexHull(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonsSymDifference(FunctionFactory & factory);
|
void registerFunctionPolygonsSymDifference(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonsEquals(FunctionFactory & factory);
|
void registerFunctionPolygonsEquals(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonsDistance(FunctionFactory & factory);
|
void registerFunctionPolygonsDistance(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonsWithin(FunctionFactory & factory);
|
void registerFunctionPolygonsWithin(FunctionFactory & factory);
|
||||||
// void registerFunctionPolygonPerimeter(FunctionFactory & factory);
|
void registerFunctionPolygonPerimeter(FunctionFactory & factory);
|
||||||
void registerFunctionGeohashEncode(FunctionFactory & factory);
|
void registerFunctionGeohashEncode(FunctionFactory & factory);
|
||||||
void registerFunctionGeohashDecode(FunctionFactory & factory);
|
void registerFunctionGeohashDecode(FunctionFactory & factory);
|
||||||
void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
||||||
void registerFunctionWkt(FunctionFactory & factory);
|
void registerFunctionWkt(FunctionFactory & factory);
|
||||||
void registerFunctionReadWkt(FunctionFactory & factory);
|
void registerFunctionReadWkt(FunctionFactory & factory);
|
||||||
void registerFunctionDescribeGeometry(FunctionFactory & factory);
|
|
||||||
void registerFunctionSvg(FunctionFactory & factory);
|
void registerFunctionSvg(FunctionFactory & factory);
|
||||||
|
|
||||||
#if USE_H3
|
#if USE_H3
|
||||||
@ -50,20 +49,19 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
|||||||
registerFunctionPointInEllipses(factory);
|
registerFunctionPointInEllipses(factory);
|
||||||
registerFunctionPointInPolygon(factory);
|
registerFunctionPointInPolygon(factory);
|
||||||
registerFunctionPolygonsIntersection(factory);
|
registerFunctionPolygonsIntersection(factory);
|
||||||
// registerFunctionPolygonsUnion(factory);
|
registerFunctionPolygonsUnion(factory);
|
||||||
registerFunctionPolygonArea(factory);
|
registerFunctionPolygonArea(factory);
|
||||||
// registerFunctionPolygonConvexHull(factory);
|
registerFunctionPolygonConvexHull(factory);
|
||||||
// registerFunctionPolygonsSymDifference(factory);
|
registerFunctionPolygonsSymDifference(factory);
|
||||||
// registerFunctionPolygonsEquals(factory);
|
registerFunctionPolygonsEquals(factory);
|
||||||
// registerFunctionPolygonsDistance(factory);
|
registerFunctionPolygonsDistance(factory);
|
||||||
// registerFunctionPolygonsWithin(factory);
|
registerFunctionPolygonsWithin(factory);
|
||||||
// registerFunctionPolygonPerimeter(factory);
|
registerFunctionPolygonPerimeter(factory);
|
||||||
registerFunctionGeohashEncode(factory);
|
registerFunctionGeohashEncode(factory);
|
||||||
registerFunctionGeohashDecode(factory);
|
registerFunctionGeohashDecode(factory);
|
||||||
registerFunctionGeohashesInBox(factory);
|
registerFunctionGeohashesInBox(factory);
|
||||||
registerFunctionWkt(factory);
|
registerFunctionWkt(factory);
|
||||||
registerFunctionReadWkt(factory);
|
registerFunctionReadWkt(factory);
|
||||||
registerFunctionDescribeGeometry(factory);
|
|
||||||
registerFunctionSvg(factory);
|
registerFunctionSvg(factory);
|
||||||
|
|
||||||
#if USE_H3
|
#if USE_H3
|
||||||
|
@ -69,10 +69,10 @@ public:
|
|||||||
|
|
||||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||||
{
|
{
|
||||||
using TypeParser = std::decay_t<decltype(type)>;
|
using TypeConverter = std::decay_t<decltype(type)>;
|
||||||
using Parser = typename TypeParser::Type;
|
using Converter = typename TypeConverter::Type;
|
||||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||||
auto figures = parser.parse();
|
auto figures = converter.convert();
|
||||||
|
|
||||||
bool has_style = arguments.size() > 1;
|
bool has_style = arguments.size() > 1;
|
||||||
ColumnPtr style;
|
ColumnPtr style;
|
||||||
|
@ -42,10 +42,10 @@ public:
|
|||||||
|
|
||||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||||
{
|
{
|
||||||
using TypeParser = std::decay_t<decltype(type)>;
|
using TypeConverter = std::decay_t<decltype(type)>;
|
||||||
using Parser = typename TypeParser::Type;
|
using Converter = typename TypeConverter::Type;
|
||||||
Parser parser(arguments[0].column->convertToFullColumnIfConst());
|
Converter converter(arguments[0].column->convertToFullColumnIfConst());
|
||||||
auto figures = parser.parse();
|
auto figures = converter.convert();
|
||||||
|
|
||||||
for (size_t i = 0; i < input_rows_count; i++)
|
for (size_t i = 0; i < input_rows_count; i++)
|
||||||
{
|
{
|
||||||
|
@ -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