mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #19257 from nikitamikhaylov/geometry_functions
Merging geometry functions
This commit is contained in:
commit
e234990cf6
@ -76,6 +76,16 @@
|
||||
# endif
|
||||
#endif
|
||||
|
||||
#if !defined(UNDEFINED_BEHAVIOR_SANITIZER)
|
||||
# if defined(__has_feature)
|
||||
# if __has_feature(undefined_behavior_sanitizer)
|
||||
# define UNDEFINED_BEHAVIOR_SANITIZER 1
|
||||
# endif
|
||||
# elif defined(__UNDEFINED_BEHAVIOR_SANITIZER__)
|
||||
# define UNDEFINED_BEHAVIOR_SANITIZER 1
|
||||
# endif
|
||||
#endif
|
||||
|
||||
#if defined(ADDRESS_SANITIZER)
|
||||
# define BOOST_USE_ASAN 1
|
||||
# define BOOST_USE_UCONTEXT 1
|
||||
|
@ -63,7 +63,7 @@ using NamesAndTypes = std::vector<NameAndTypePair>;
|
||||
class NamesAndTypesList : public std::list<NameAndTypePair>
|
||||
{
|
||||
public:
|
||||
NamesAndTypesList() {}
|
||||
NamesAndTypesList() = default;
|
||||
|
||||
NamesAndTypesList(std::initializer_list<NameAndTypePair> init) : std::list<NameAndTypePair>(init) {}
|
||||
|
||||
|
@ -1,3 +1,6 @@
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
@ -11,88 +14,84 @@ namespace DB
|
||||
|
||||
namespace
|
||||
{
|
||||
const auto point_data_type = std::make_shared<const DataTypeTuple>(
|
||||
DataTypes{std::make_shared<const DataTypeFloat64>(), std::make_shared<const DataTypeFloat64>()}
|
||||
);
|
||||
|
||||
class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
const auto ring_data_type = std::make_shared<const DataTypeArray>(DataTypeCustomPointSerialization::nestedDataType());
|
||||
|
||||
const auto polygon_data_type = std::make_shared<const DataTypeArray>(DataTypeCustomRingSerialization::nestedDataType());
|
||||
|
||||
const auto multipolygon_data_type = std::make_shared<const DataTypeArray>(DataTypeCustomPolygonSerialization::nestedDataType());
|
||||
}
|
||||
|
||||
|
||||
void DataTypeCustomPointSerialization::serializeText(
|
||||
const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeTuple>(
|
||||
DataTypes({std::make_unique<DataTypeFloat64>(), std::make_unique<DataTypeFloat64>()})));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
void DataTypeCustomPointSerialization::deserializeText(
|
||||
IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomPointSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
DataTypePtr DataTypeCustomPointSerialization::nestedDataType()
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
return point_data_type;
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomRingSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
void DataTypeCustomRingSerialization::serializeText(
|
||||
const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
void DataTypeCustomRingSerialization::deserializeText(
|
||||
IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomPolygonSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
DataTypePtr DataTypeCustomRingSerialization::nestedDataType()
|
||||
{
|
||||
return ring_data_type;
|
||||
}
|
||||
|
||||
void DataTypeCustomPolygonSerialization::serializeText(
|
||||
const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeCustomPolygonSerialization::deserializeText(
|
||||
IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeCustomPolygonSerialization::nestedDataType()
|
||||
{
|
||||
return polygon_data_type;
|
||||
}
|
||||
|
||||
void DataTypeCustomMultiPolygonSerialization::serializeText(
|
||||
const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeCustomMultiPolygonSerialization::deserializeText(
|
||||
IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeCustomMultiPolygonSerialization::nestedDataType()
|
||||
{
|
||||
return multipolygon_data_type;
|
||||
}
|
||||
|
||||
void registerDataTypeDomainGeo(DataTypeFactory & factory)
|
||||
|
56
src/DataTypes/DataTypeCustomGeo.h
Normal file
56
src/DataTypes/DataTypeCustomGeo.h
Normal file
@ -0,0 +1,56 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleTextSerialization.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
static DataTypePtr nestedDataType();
|
||||
};
|
||||
|
||||
|
||||
class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
static DataTypePtr nestedDataType();
|
||||
};
|
||||
|
||||
class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
static DataTypePtr nestedDataType();
|
||||
};
|
||||
|
||||
class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
static DataTypePtr nestedDataType();
|
||||
};
|
||||
|
||||
}
|
@ -9,8 +9,7 @@ namespace DB
|
||||
class DataTypeWithSimpleSerialization : public IDataType
|
||||
{
|
||||
protected:
|
||||
DataTypeWithSimpleSerialization()
|
||||
{}
|
||||
DataTypeWithSimpleSerialization() = default;
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
|
360
src/Functions/geometryConverters.h
Normal file
360
src/Functions/geometryConverters.h
Normal file
@ -0,0 +1,360 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
#include <boost/geometry/geometries/geometries.hpp>
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeCustomGeo.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <cmath>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
using Ring = boost::geometry::model::ring<Point>;
|
||||
|
||||
template <typename Point>
|
||||
using Polygon = boost::geometry::model::polygon<Point>;
|
||||
|
||||
template <typename Point>
|
||||
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon<Point>>;
|
||||
|
||||
using CartesianPoint = boost::geometry::model::d2::point_xy<Float64>;
|
||||
using CartesianRing = Ring<CartesianPoint>;
|
||||
using CartesianPolygon = Polygon<CartesianPoint>;
|
||||
using CartesianMultiPolygon = MultiPolygon<CartesianPoint>;
|
||||
|
||||
using SphericalPoint = boost::geometry::model::point<Float64, 2, boost::geometry::cs::spherical_equatorial<boost::geometry::degree>>;
|
||||
using SphericalRing = Ring<SphericalPoint>;
|
||||
using SphericalPolygon = Polygon<SphericalPoint>;
|
||||
using SphericalMultiPolygon = MultiPolygon<SphericalPoint>;
|
||||
|
||||
/**
|
||||
* Class which takes converts Column with type Tuple(Float64, Float64) to a vector of boost point type.
|
||||
* They are (x,y) in case of cartesian coordinated and (lon,lat) in case of Spherical.
|
||||
*/
|
||||
template <typename Point>
|
||||
struct ColumnToPointsConverter
|
||||
{
|
||||
static std::vector<Point> convert(ColumnPtr col)
|
||||
{
|
||||
const auto * tuple = typeid_cast<const ColumnTuple *>(col.get());
|
||||
const auto & tuple_columns = tuple->getColumns();
|
||||
|
||||
const auto * x_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[0].get());
|
||||
const auto * y_data = typeid_cast<const ColumnFloat64 *>(tuple_columns[1].get());
|
||||
|
||||
const auto * first_container = x_data->getData().data();
|
||||
const auto * second_container = y_data->getData().data();
|
||||
|
||||
std::vector<Point> answer(col->size());
|
||||
|
||||
for (size_t i = 0; i < col->size(); ++i)
|
||||
{
|
||||
const Float64 first = first_container[i];
|
||||
const Float64 second = second_container[i];
|
||||
|
||||
if (isNaN(first) || isNaN(second))
|
||||
throw Exception("Point's component must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (isinf(first) || isinf(second))
|
||||
throw Exception("Point's component must not be infinite", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
answer[i] = Point(first, second);
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Point>
|
||||
struct ColumnToRingsConverter
|
||||
{
|
||||
static std::vector<Ring<Point>> convert(ColumnPtr col)
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
size_t prev_offset = 0;
|
||||
std::vector<Ring<Point>> answer;
|
||||
answer.reserve(offsets.size());
|
||||
auto tmp = ColumnToPointsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
||||
for (size_t offset : offsets)
|
||||
{
|
||||
answer.emplace_back(tmp.begin() + prev_offset, tmp.begin() + offset);
|
||||
prev_offset = offset;
|
||||
}
|
||||
return answer;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Point>
|
||||
struct ColumnToPolygonsConverter
|
||||
{
|
||||
static std::vector<Polygon<Point>> convert(ColumnPtr col)
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
std::vector<Polygon<Point>> answer(offsets.size());
|
||||
auto all_rings = ColumnToRingsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
||||
|
||||
size_t prev_offset = 0;
|
||||
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
||||
{
|
||||
const auto current_array_size = offsets[iter] - prev_offset;
|
||||
answer[iter].outer() = std::move(all_rings[prev_offset]);
|
||||
answer[iter].inners().reserve(current_array_size);
|
||||
for (size_t inner_holes = prev_offset + 1; inner_holes < offsets[iter]; ++inner_holes)
|
||||
answer[iter].inners().emplace_back(std::move(all_rings[inner_holes]));
|
||||
prev_offset = offsets[iter];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Point>
|
||||
struct ColumnToMultiPolygonsConverter
|
||||
{
|
||||
static std::vector<MultiPolygon<Point>> convert(ColumnPtr col)
|
||||
{
|
||||
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
|
||||
size_t prev_offset = 0;
|
||||
std::vector<MultiPolygon<Point>> answer(offsets.size());
|
||||
|
||||
auto all_polygons = ColumnToPolygonsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
|
||||
|
||||
for (size_t iter = 0; iter < offsets.size(); ++iter)
|
||||
{
|
||||
for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter)
|
||||
answer[iter].emplace_back(std::move(all_polygons[polygon_iter]));
|
||||
prev_offset = offsets[iter];
|
||||
}
|
||||
|
||||
return answer;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// To serialize Spherical or Cartesian point (a pair of numbers in both cases).
|
||||
template <typename Point>
|
||||
class PointSerializer
|
||||
{
|
||||
public:
|
||||
PointSerializer()
|
||||
: first(ColumnFloat64::create())
|
||||
, second(ColumnFloat64::create())
|
||||
, first_container(first->getData())
|
||||
, second_container(second->getData())
|
||||
{}
|
||||
|
||||
explicit PointSerializer(size_t n)
|
||||
: first(ColumnFloat64::create(n))
|
||||
, second(ColumnFloat64::create(n))
|
||||
, first_container(first->getData())
|
||||
, second_container(second->getData())
|
||||
{}
|
||||
|
||||
void add(const Point & point)
|
||||
{
|
||||
first_container.emplace_back(point.template get<0>());
|
||||
second_container.emplace_back(point.template get<1>());
|
||||
}
|
||||
|
||||
ColumnPtr finalize()
|
||||
{
|
||||
Columns columns(2);
|
||||
columns[0] = std::move(first);
|
||||
columns[1] = std::move(second);
|
||||
|
||||
return ColumnTuple::create(columns);
|
||||
}
|
||||
|
||||
private:
|
||||
ColumnFloat64::MutablePtr first;
|
||||
ColumnFloat64::MutablePtr second;
|
||||
|
||||
ColumnFloat64::Container & first_container;
|
||||
ColumnFloat64::Container & second_container;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring as Ring
|
||||
template <typename Point>
|
||||
class RingSerializer
|
||||
{
|
||||
public:
|
||||
RingSerializer()
|
||||
: offsets(ColumnUInt64::create())
|
||||
{}
|
||||
|
||||
explicit RingSerializer(size_t n)
|
||||
: offsets(ColumnUInt64::create(n))
|
||||
{}
|
||||
|
||||
void add(const Ring<Point> & ring)
|
||||
{
|
||||
size += ring.size();
|
||||
offsets->insertValue(size);
|
||||
for (const auto & point : ring)
|
||||
point_serializer.add(point);
|
||||
}
|
||||
|
||||
ColumnPtr finalize()
|
||||
{
|
||||
return ColumnArray::create(point_serializer.finalize(), std::move(offsets));
|
||||
}
|
||||
|
||||
private:
|
||||
size_t size = 0;
|
||||
PointSerializer<Point> point_serializer;
|
||||
ColumnUInt64::MutablePtr offsets;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring, Polygon as Polygon
|
||||
template <typename Point>
|
||||
class PolygonSerializer
|
||||
{
|
||||
public:
|
||||
PolygonSerializer()
|
||||
: offsets(ColumnUInt64::create())
|
||||
{}
|
||||
|
||||
explicit PolygonSerializer(size_t n)
|
||||
: offsets(ColumnUInt64::create(n))
|
||||
{}
|
||||
|
||||
void add(const Ring<Point> & ring)
|
||||
{
|
||||
size++;
|
||||
offsets->insertValue(size);
|
||||
ring_serializer.add(ring);
|
||||
}
|
||||
|
||||
void add(const Polygon<Point> & polygon)
|
||||
{
|
||||
/// Outer ring + all inner rings (holes).
|
||||
size += 1 + polygon.inners().size();
|
||||
offsets->insertValue(size);
|
||||
ring_serializer.add(polygon.outer());
|
||||
for (const auto & ring : polygon.inners())
|
||||
ring_serializer.add(ring);
|
||||
}
|
||||
|
||||
ColumnPtr finalize()
|
||||
{
|
||||
return ColumnArray::create(ring_serializer.finalize(), std::move(offsets));
|
||||
}
|
||||
|
||||
private:
|
||||
size_t size = 0;
|
||||
RingSerializer<Point> ring_serializer;
|
||||
ColumnUInt64::MutablePtr offsets;
|
||||
};
|
||||
|
||||
/// Serialize Point, Ring, Polygon, MultiPolygon as MultiPolygon
|
||||
template <typename Point>
|
||||
class MultiPolygonSerializer
|
||||
{
|
||||
public:
|
||||
MultiPolygonSerializer()
|
||||
: offsets(ColumnUInt64::create())
|
||||
{}
|
||||
|
||||
explicit MultiPolygonSerializer(size_t n)
|
||||
: offsets(ColumnUInt64::create(n))
|
||||
{}
|
||||
|
||||
void add(const Ring<Point> & ring)
|
||||
{
|
||||
size++;
|
||||
offsets->insertValue(size);
|
||||
polygon_serializer.add(ring);
|
||||
}
|
||||
|
||||
void add(const Polygon<Point> & polygon)
|
||||
{
|
||||
size++;
|
||||
offsets->insertValue(size);
|
||||
polygon_serializer.add(polygon);
|
||||
}
|
||||
|
||||
void add(const MultiPolygon<Point> & multi_polygon)
|
||||
{
|
||||
size += multi_polygon.size();
|
||||
offsets->insertValue(size);
|
||||
for (const auto & polygon : multi_polygon)
|
||||
{
|
||||
polygon_serializer.add(polygon);
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr finalize()
|
||||
{
|
||||
return ColumnArray::create(polygon_serializer.finalize(), std::move(offsets));
|
||||
}
|
||||
|
||||
private:
|
||||
size_t size = 0;
|
||||
PolygonSerializer<Point> polygon_serializer;
|
||||
ColumnUInt64::MutablePtr offsets;
|
||||
};
|
||||
|
||||
|
||||
template <typename PType>
|
||||
struct ConverterType
|
||||
{
|
||||
using Type = PType;
|
||||
};
|
||||
|
||||
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 (DataTypeCustomPointSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<ColumnToPointsConverter<Point>>());
|
||||
else if (DataTypeCustomRingSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<ColumnToRingsConverter<Point>>());
|
||||
else if (DataTypeCustomPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<ColumnToPolygonsConverter<Point>>());
|
||||
else if (DataTypeCustomMultiPolygonSerialization::nestedDataType()->equals(*type))
|
||||
return f(ConverterType<ColumnToMultiPolygonsConverter<Point>>());
|
||||
throw Exception(fmt::format("Unknown geometry type {}", type->getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
|
||||
template <typename Point, typename F>
|
||||
static void callOnTwoGeometryDataTypes(DataTypePtr left_type, DataTypePtr right_type, F && func)
|
||||
{
|
||||
return callOnGeometryDataType<Point>(left_type, [&](const auto & left_types)
|
||||
{
|
||||
using LeftConverterType = std::decay_t<decltype(left_types)>;
|
||||
|
||||
return callOnGeometryDataType<Point>(right_type, [&](const auto & right_types)
|
||||
{
|
||||
using RightConverterType = std::decay_t<decltype(right_types)>;
|
||||
|
||||
return func(LeftConverterType(), RightConverterType());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
}
|
@ -13,6 +13,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/ObjectPool.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <common/arithmeticOverflow.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
@ -425,7 +426,14 @@ private:
|
||||
{
|
||||
out_container.reserve(end - begin);
|
||||
for (size_t i = begin; i < end; ++i)
|
||||
{
|
||||
Int64 result = 0;
|
||||
if (common::mulOverflow(static_cast<Int64>(x_data[i]), static_cast<Int64>(y_data[i]), result))
|
||||
throw Exception("The coordinates of the point are such that subsequent calculations cannot be performed correctly. " \
|
||||
"Most likely they are very large in modulus.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
out_container.emplace_back(x_data[i], y_data[i]);
|
||||
}
|
||||
}
|
||||
|
||||
void parseConstPolygonWithoutHolesFromSingleColumn(const IColumn & column, size_t i, Polygon & out_polygon) const
|
||||
@ -540,7 +548,7 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
void parseConstPolygon(const ColumnsWithTypeAndName & arguments, Polygon & out_polygon) const
|
||||
void NO_SANITIZE_UNDEFINED parseConstPolygon(const ColumnsWithTypeAndName & arguments, Polygon & out_polygon) const
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
parseConstPolygonFromSingleColumn(arguments, out_polygon);
|
||||
|
107
src/Functions/polygonArea.cpp
Normal file
107
src/Functions/polygonArea.cpp
Normal file
@ -0,0 +1,107 @@
|
||||
#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 <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 <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonArea : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
explicit FunctionPolygonArea() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonArea>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
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 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;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, Converter>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::area(geometries[i]));
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonArea<CartesianPoint>::name = "polygonAreaCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonArea<SphericalPoint>::name = "polygonAreaSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonArea(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonArea<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonArea<SphericalPoint>>();
|
||||
}
|
||||
|
||||
|
||||
}
|
105
src/Functions/polygonConvexHull.cpp
Normal file
105
src/Functions/polygonConvexHull.cpp
Normal file
@ -0,0 +1,105 @@
|
||||
#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 <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 <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonConvexHull : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
explicit FunctionPolygonConvexHull() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonConvexHull>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return DataTypeCustomPolygonSerialization::nestedDataType();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
PolygonSerializer<Point> serializer;
|
||||
|
||||
callOnGeometryDataType<Point>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
if constexpr (std::is_same_v<Converter, ColumnToPointsConverter<Point>>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be a Point", getName()), ErrorCodes::BAD_ARGUMENTS);
|
||||
else
|
||||
{
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
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";
|
||||
|
||||
|
||||
void registerFunctionPolygonConvexHull(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonConvexHull<CartesianPoint>>();
|
||||
}
|
||||
|
||||
}
|
107
src/Functions/polygonPerimeter.cpp
Normal file
107
src/Functions/polygonPerimeter.cpp
Normal file
@ -0,0 +1,107 @@
|
||||
#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 <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 <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonPerimeter : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
explicit FunctionPolygonPerimeter() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonPerimeter>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
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 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;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, Converter>)
|
||||
throw Exception(fmt::format("The argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
res_data.emplace_back(boost::geometry::perimeter(geometries[i]));
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonPerimeter<CartesianPoint>::name = "polygonPerimeterCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonPerimeter<SphericalPoint>::name = "polygonPerimeterSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonPerimeter(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonPerimeter<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonPerimeter<SphericalPoint>>();
|
||||
}
|
||||
|
||||
|
||||
}
|
117
src/Functions/polygonsDistance.cpp
Normal file
117
src/Functions/polygonsDistance.cpp
Normal file
@ -0,0 +1,117 @@
|
||||
#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 <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 <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsDistance : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
explicit FunctionPolygonsDistance() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsDistance>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
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 res_column = ColumnFloat64::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
res_data.emplace_back(boost::geometry::distance(first[i], second[i]));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsDistance<CartesianPoint>::name = "polygonsDistanceCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsDistance<SphericalPoint>::name = "polygonsDistanceSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonsDistance(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsDistance<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsDistance<SphericalPoint>>();
|
||||
}
|
||||
|
||||
|
||||
}
|
114
src/Functions/polygonsEquals.cpp
Normal file
114
src/Functions/polygonsEquals.cpp
Normal file
@ -0,0 +1,114 @@
|
||||
#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 <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 <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsEquals : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
explicit FunctionPolygonsEquals() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsEquals>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
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 res_column = ColumnUInt8::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
/// Main work here.
|
||||
res_data.emplace_back(boost::geometry::equals(first[i], second[i]));
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsEquals<CartesianPoint>::name = "polygonsEqualsCartesian";
|
||||
|
||||
|
||||
void registerFunctionPolygonsEquals(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsEquals<CartesianPoint>>();
|
||||
}
|
||||
|
||||
}
|
122
src/Functions/polygonsIntersection.cpp
Normal file
122
src/Functions/polygonsIntersection.cpp
Normal file
@ -0,0 +1,122 @@
|
||||
#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 <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 <memory>
|
||||
#include <utility>
|
||||
#include <chrono>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsIntersection : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
explicit FunctionPolygonsIntersection() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsIntersection>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
/// Intersection of each with figure with each could be easily represent as MultiPolygon.
|
||||
return DataTypeCustomMultiPolygonSerialization::nestedDataType();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
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> intersection{};
|
||||
/// Main work here.
|
||||
boost::geometry::intersection(first[i], second[i], intersection);
|
||||
|
||||
serializer.add(intersection);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsIntersection<CartesianPoint>::name = "polygonsIntersectionCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsIntersection<SphericalPoint>::name = "polygonsIntersectionSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonsIntersection(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsIntersection<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsIntersection<SphericalPoint>>();
|
||||
}
|
||||
|
||||
}
|
116
src/Functions/polygonsSymDifference.cpp
Normal file
116
src/Functions/polygonsSymDifference.cpp
Normal file
@ -0,0 +1,116 @@
|
||||
#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 <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 <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsSymDifference : public IFunction
|
||||
{
|
||||
public:
|
||||
static const char * name;
|
||||
|
||||
explicit FunctionPolygonsSymDifference() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsSymDifference>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
MultiPolygon<Point> sym_difference{};
|
||||
boost::geometry::sym_difference(first[i], second[i], sym_difference);
|
||||
|
||||
serializer.add(sym_difference);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsSymDifference<CartesianPoint>::name = "polygonsSymDifferenceCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsSymDifference<SphericalPoint>::name = "polygonsSymDifferenceSpherical";
|
||||
|
||||
void registerFunctionPolygonsSymDifference(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsSymDifference<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsSymDifference<SphericalPoint>>();
|
||||
}
|
||||
|
||||
}
|
120
src/Functions/polygonsUnion.cpp
Normal file
120
src/Functions/polygonsUnion.cpp
Normal file
@ -0,0 +1,120 @@
|
||||
#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 <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 <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsUnion : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
explicit FunctionPolygonsUnion() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsUnion>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
MultiPolygonSerializer<Point> serializer;
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// We are not interested in some pitfalls in third-party libraries
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
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{};
|
||||
/// Main work here.
|
||||
boost::geometry::union_(first[i], second[i], polygons_union);
|
||||
|
||||
serializer.add(polygons_union);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsUnion<CartesianPoint>::name = "polygonsUnionCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsUnion<SphericalPoint>::name = "polygonsUnionSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonsUnion(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsUnion<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsUnion<SphericalPoint>>();
|
||||
}
|
||||
|
||||
}
|
119
src/Functions/polygonsWithin.cpp
Normal file
119
src/Functions/polygonsWithin.cpp
Normal file
@ -0,0 +1,119 @@
|
||||
#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 <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 <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Point>
|
||||
class FunctionPolygonsWithin : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name;
|
||||
|
||||
explicit FunctionPolygonsWithin() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionPolygonsWithin>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
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 res_column = ColumnUInt8::create();
|
||||
auto & res_data = res_column->getData();
|
||||
res_data.reserve(input_rows_count);
|
||||
|
||||
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 LeftConverter = typename LeftConverterType::Type;
|
||||
using RightConverter = typename RightConverterType::Type;
|
||||
|
||||
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
|
||||
throw Exception(fmt::format("Any argument of function {} must not be Point", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst());
|
||||
|
||||
/// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign)
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
boost::geometry::correct(first[i]);
|
||||
boost::geometry::correct(second[i]);
|
||||
|
||||
res_data.emplace_back(boost::geometry::within(first[i], second[i]));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsWithin<CartesianPoint>::name = "polygonsWithinCartesian";
|
||||
|
||||
template <>
|
||||
const char * FunctionPolygonsWithin<SphericalPoint>::name = "polygonsWithinSpherical";
|
||||
|
||||
|
||||
void registerFunctionPolygonsWithin(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPolygonsWithin<CartesianPoint>>();
|
||||
factory.registerFunction<FunctionPolygonsWithin<SphericalPoint>>();
|
||||
}
|
||||
|
||||
}
|
105
src/Functions/readWkt.cpp
Normal file
105
src/Functions/readWkt.cpp
Normal file
@ -0,0 +1,105 @@
|
||||
#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;
|
||||
}
|
||||
|
||||
|
||||
template <class DataType, class Geometry, class Serializer, class NameHolder>
|
||||
class FunctionReadWkt : public IFunction
|
||||
{
|
||||
public:
|
||||
explicit FunctionReadWkt() = default;
|
||||
|
||||
static constexpr const char * name = NameHolder::name;
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
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 DataType::nestedDataType();
|
||||
}
|
||||
|
||||
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());
|
||||
|
||||
Serializer serializer;
|
||||
Geometry geometry;
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
const auto & str = column_string->getDataAt(i).toString();
|
||||
boost::geometry::read_wkt(str, geometry);
|
||||
serializer.add(geometry);
|
||||
}
|
||||
|
||||
return serializer.finalize();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReadWkt<DataType, Geometry, Serializer, NameHolder>>();
|
||||
}
|
||||
};
|
||||
|
||||
struct ReadWktPointNameHolder
|
||||
{
|
||||
static constexpr const char * name = "readWktPoint";
|
||||
};
|
||||
|
||||
struct ReadWktRingNameHolder
|
||||
{
|
||||
static constexpr const char * name = "readWktRing";
|
||||
};
|
||||
|
||||
struct ReadWktPolygonNameHolder
|
||||
{
|
||||
static constexpr const char * name = "readWktPolygon";
|
||||
};
|
||||
|
||||
struct ReadWktMultiPolygonNameHolder
|
||||
{
|
||||
static constexpr const char * name = "readWktMultiPolygon";
|
||||
};
|
||||
|
||||
void registerFunctionReadWkt(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomPointSerialization, CartesianPoint, PointSerializer<CartesianPoint>, ReadWktPointNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomRingSerialization, CartesianRing, RingSerializer<CartesianPoint>, ReadWktRingNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomPolygonSerialization, CartesianPolygon, PolygonSerializer<CartesianPoint>, ReadWktPolygonNameHolder>>();
|
||||
factory.registerFunction<FunctionReadWkt<DataTypeCustomMultiPolygonSerialization, CartesianMultiPolygon, MultiPolygonSerializer<CartesianPoint>, ReadWktMultiPolygonNameHolder>>();
|
||||
}
|
||||
|
||||
}
|
@ -10,9 +10,21 @@ class FunctionFactory;
|
||||
void registerFunctionGeoDistance(FunctionFactory & factory);
|
||||
void registerFunctionPointInEllipses(FunctionFactory & factory);
|
||||
void registerFunctionPointInPolygon(FunctionFactory & factory);
|
||||
void registerFunctionPolygonsIntersection(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 registerFunctionGeohashEncode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashDecode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
||||
void registerFunctionWkt(FunctionFactory & factory);
|
||||
void registerFunctionReadWkt(FunctionFactory & factory);
|
||||
void registerFunctionSvg(FunctionFactory & factory);
|
||||
|
||||
#if USE_H3
|
||||
void registerFunctionGeoToH3(FunctionFactory &);
|
||||
@ -36,9 +48,21 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
||||
registerFunctionGeoDistance(factory);
|
||||
registerFunctionPointInEllipses(factory);
|
||||
registerFunctionPointInPolygon(factory);
|
||||
registerFunctionPolygonsIntersection(factory);
|
||||
registerFunctionPolygonsUnion(factory);
|
||||
registerFunctionPolygonArea(factory);
|
||||
registerFunctionPolygonConvexHull(factory);
|
||||
registerFunctionPolygonsSymDifference(factory);
|
||||
registerFunctionPolygonsEquals(factory);
|
||||
registerFunctionPolygonsDistance(factory);
|
||||
registerFunctionPolygonsWithin(factory);
|
||||
registerFunctionPolygonPerimeter(factory);
|
||||
registerFunctionGeohashEncode(factory);
|
||||
registerFunctionGeohashDecode(factory);
|
||||
registerFunctionGeohashesInBox(factory);
|
||||
registerFunctionWkt(factory);
|
||||
registerFunctionReadWkt(factory);
|
||||
registerFunctionSvg(factory);
|
||||
|
||||
#if USE_H3
|
||||
registerFunctionGeoToH3(factory);
|
||||
|
105
src/Functions/svg.cpp
Normal file
105
src/Functions/svg.cpp
Normal file
@ -0,0 +1,105 @@
|
||||
#include <DataTypes/DataTypeString.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;
|
||||
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
|
||||
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
||||
}
|
||||
|
||||
class FunctionSvg : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "svg";
|
||||
|
||||
explicit FunctionSvg() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionSvg>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() > 2)
|
||||
{
|
||||
throw Exception("Too many arguments", ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION);
|
||||
}
|
||||
else if (arguments.empty())
|
||||
{
|
||||
throw Exception("Too few arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
|
||||
}
|
||||
else if (arguments.size() == 2 && checkAndGetDataType<DataTypeString>(arguments[1].get()) == nullptr)
|
||||
{
|
||||
throw Exception("Second argument should be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnString::create();
|
||||
bool has_style = arguments.size() > 1;
|
||||
ColumnPtr style;
|
||||
if (has_style)
|
||||
style = arguments[1].column;
|
||||
|
||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
std::stringstream str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
boost::geometry::correct(figures[i]);
|
||||
str << boost::geometry::svg(figures[i], has_style ? style->getDataAt(i).toString() : "");
|
||||
std::string serialized = str.str();
|
||||
res_column->insertData(serialized.c_str(), serialized.size());
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionSvg(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionSvg>();
|
||||
}
|
||||
|
||||
}
|
74
src/Functions/wkt.cpp
Normal file
74
src/Functions/wkt.cpp
Normal file
@ -0,0 +1,74 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/geometryConverters.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionWkt : public IFunction
|
||||
{
|
||||
public:
|
||||
static inline const char * name = "wkt";
|
||||
|
||||
explicit FunctionWkt() = default;
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionWkt>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||
{
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
|
||||
{
|
||||
auto res_column = ColumnString::create();
|
||||
|
||||
callOnGeometryDataType<CartesianPoint>(arguments[0].type, [&] (const auto & type)
|
||||
{
|
||||
using TypeConverter = std::decay_t<decltype(type)>;
|
||||
using Converter = typename TypeConverter::Type;
|
||||
|
||||
auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst());
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; i++)
|
||||
{
|
||||
std::stringstream str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
str << boost::geometry::wkt(figures[i]);
|
||||
std::string serialized = str.str();
|
||||
res_column->insertData(serialized.c_str(), serialized.size());
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return res_column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionWkt(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionWkt>();
|
||||
}
|
||||
|
||||
}
|
@ -377,6 +377,15 @@ SRCS(
|
||||
plus.cpp
|
||||
pointInEllipses.cpp
|
||||
pointInPolygon.cpp
|
||||
polygonArea.cpp
|
||||
polygonConvexHull.cpp
|
||||
polygonPerimeter.cpp
|
||||
polygonsDistance.cpp
|
||||
polygonsEquals.cpp
|
||||
polygonsIntersection.cpp
|
||||
polygonsSymDifference.cpp
|
||||
polygonsUnion.cpp
|
||||
polygonsWithin.cpp
|
||||
position.cpp
|
||||
positionCaseInsensitive.cpp
|
||||
positionCaseInsensitiveUTF8.cpp
|
||||
@ -389,6 +398,7 @@ SRCS(
|
||||
randomPrintableASCII.cpp
|
||||
randomString.cpp
|
||||
randomStringUTF8.cpp
|
||||
readWkt.cpp
|
||||
regexpQuoteMeta.cpp
|
||||
registerFunctions.cpp
|
||||
registerFunctionsArithmetic.cpp
|
||||
@ -447,6 +457,7 @@ SRCS(
|
||||
subtractSeconds.cpp
|
||||
subtractWeeks.cpp
|
||||
subtractYears.cpp
|
||||
svg.cpp
|
||||
tan.cpp
|
||||
tanh.cpp
|
||||
tcpPort.cpp
|
||||
@ -525,6 +536,7 @@ SRCS(
|
||||
visitParamExtractString.cpp
|
||||
visitParamExtractUInt.cpp
|
||||
visitParamHas.cpp
|
||||
wkt.cpp
|
||||
yandexConsistentHash.cpp
|
||||
yesterday.cpp
|
||||
|
||||
|
@ -45,7 +45,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
WriteBufferFromVector(VectorType & vector_)
|
||||
explicit WriteBufferFromVector(VectorType & vector_)
|
||||
: WriteBuffer(reinterpret_cast<Position>(vector_.data()), vector_.size()), vector(vector_)
|
||||
{
|
||||
if (vector.empty())
|
||||
|
@ -0,0 +1 @@
|
||||
[[(0,0),(0,5),(5,5),(5,0),(0,0)]]
|
1
tests/queries/0_stateless/01300_polygon_convex_hull.sql
Normal file
1
tests/queries/0_stateless/01300_polygon_convex_hull.sql
Normal file
@ -0,0 +1 @@
|
||||
select polygonConvexHullCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (2., 3.)]]]);
|
16
tests/queries/0_stateless/01300_read_wkt.reference
Normal file
16
tests/queries/0_stateless/01300_read_wkt.reference
Normal file
@ -0,0 +1,16 @@
|
||||
(0,0)
|
||||
[[(1,0),(10,0),(10,10),(0,10),(1,0)]]
|
||||
[[(0,0),(10,0),(10,10),(0,10),(0,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]]
|
||||
[[[(2,0),(10,0),(10,10),(0,10),(2,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]],[[(-10,-10),(-10,-9),(-9,10),(-10,-10)]]]
|
||||
(0,0)
|
||||
(1,0)
|
||||
(2,0)
|
||||
[[(1,0),(10,0),(10,10),(0,10),(1,0)]]
|
||||
[[(0,0),(10,0),(10,10),(0,10),(0,0)]]
|
||||
[[(2,0),(10,0),(10,10),(0,10),(2,0)]]
|
||||
[[(0,0),(10,0),(10,10),(0,10),(0,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]]
|
||||
[[(2,0),(10,0),(10,10),(0,10),(2,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]]
|
||||
[[(1,0),(10,0),(10,10),(0,10),(1,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]]
|
||||
[[[(1,0),(10,0),(10,10),(0,10),(1,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]],[[(-10,-10),(-10,-9),(-9,10),(-10,-10)]]]
|
||||
[[[(0,0),(10,0),(10,10),(0,10),(0,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]],[[(-10,-10),(-10,-9),(-9,10),(-10,-10)]]]
|
||||
[[[(2,0),(10,0),(10,10),(0,10),(2,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]],[[(-10,-10),(-10,-9),(-9,10),(-10,-10)]]]
|
28
tests/queries/0_stateless/01300_read_wkt.sql
Normal file
28
tests/queries/0_stateless/01300_read_wkt.sql
Normal file
@ -0,0 +1,28 @@
|
||||
SELECT readWktPoint('POINT(0 0)');
|
||||
SELECT readWktPolygon('POLYGON((1 0,10 0,10 10,0 10,1 0))');
|
||||
SELECT readWktPolygon('POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))');
|
||||
SELECT readWktMultiPolygon('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))');
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ('POINT(0 0)', 1);
|
||||
INSERT INTO geo VALUES ('POINT(1 0)', 2);
|
||||
INSERT INTO geo VALUES ('POINT(2 0)', 3);
|
||||
SELECT readWktPoint(s) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ('POLYGON((1 0,10 0,10 10,0 10,1 0))', 1);
|
||||
INSERT INTO geo VALUES ('POLYGON((0 0,10 0,10 10,0 10,0 0))', 2);
|
||||
INSERT INTO geo VALUES ('POLYGON((2 0,10 0,10 10,0 10,2 0))', 3);
|
||||
INSERT INTO geo VALUES ('POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))', 4);
|
||||
INSERT INTO geo VALUES ('POLYGON((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4))', 5);
|
||||
INSERT INTO geo VALUES ('POLYGON((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4))', 6);
|
||||
SELECT readWktPolygon(s) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ('MULTIPOLYGON(((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 1);
|
||||
INSERT INTO geo VALUES ('MULTIPOLYGON(((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 2);
|
||||
INSERT INTO geo VALUES ('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 3);
|
||||
SELECT readWktMultiPolygon(s) FROM geo ORDER BY id;
|
56
tests/queries/0_stateless/01300_svg.reference
Normal file
56
tests/queries/0_stateless/01300_svg.reference
Normal file
@ -0,0 +1,56 @@
|
||||
<circle cx="0" cy="0" r="5" style=""/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style=""/>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style=""/></g>
|
||||
<circle cx="0" cy="0" r="5" style="b"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="b"/>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<circle cx="0" cy="0" r="5" style=""/>
|
||||
<circle cx="1" cy="0" r="5" style=""/>
|
||||
<circle cx="2" cy="0" r="5" style=""/>
|
||||
<circle cx="0" cy="0" r="5" style="b"/>
|
||||
<circle cx="1" cy="0" r="5" style="b"/>
|
||||
<circle cx="2" cy="0" r="5" style="b"/>
|
||||
<circle cx="0" cy="0" r="5" style="b"/>
|
||||
<circle cx="0" cy="0" r="5" style="c"/>
|
||||
<circle cx="0" cy="0" r="5" style="d"/>
|
||||
<circle cx="0" cy="0" r="5" style="b"/>
|
||||
<circle cx="1" cy="0" r="5" style="c"/>
|
||||
<circle cx="2" cy="0" r="5" style="d"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style=""/>
|
||||
<polygon points="1,0 0,10 10,10 10,0 1,0" style=""/>
|
||||
<polygon points="2,0 0,10 10,10 10,0 2,0" style=""/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="b"/>
|
||||
<polygon points="1,0 0,10 10,10 10,0 1,0" style="b"/>
|
||||
<polygon points="2,0 0,10 10,10 10,0 2,0" style="b"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="b"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="c"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="d"/>
|
||||
<polygon points="0,0 0,10 10,10 10,0 0,0" style="b"/>
|
||||
<polygon points="1,0 0,10 10,10 10,0 1,0" style="c"/>
|
||||
<polygon points="2,0 0,10 10,10 10,0 2,0" style="d"/>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="c"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="d"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="c"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="d"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style=""/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style=""/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="c"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="c"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="d"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="d"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 0,0 L 0,10 L 10,10 L 10,0 L 0,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="b"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="b"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 1,0 L 0,10 L 10,10 L 10,0 L 1,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="c"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="c"/></g>
|
||||
<g fill-rule="evenodd"><path d="M 2,0 L 0,10 L 10,10 L 10,0 L 2,0M 4,4 L 5,4 L 5,5 L 4,5 L 4,4 z " style="d"/></g><g fill-rule="evenodd"><path d="M -10,-10 L -10,-9 L -9,10 L -10,-10 z " style="d"/></g>
|
48
tests/queries/0_stateless/01300_svg.sql
Normal file
48
tests/queries/0_stateless/01300_svg.sql
Normal file
@ -0,0 +1,48 @@
|
||||
SELECT svg((0., 0.));
|
||||
SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)]);
|
||||
SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]]);
|
||||
SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]]);
|
||||
SELECT svg((0., 0.), 'b');
|
||||
SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)], 'b');
|
||||
SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], 'b');
|
||||
SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'b');
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Tuple(Float64, Float64), s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ((0., 0.), 'b', 1);
|
||||
INSERT INTO geo VALUES ((1., 0.), 'c', 2);
|
||||
INSERT INTO geo VALUES ((2., 0.), 'd', 3);
|
||||
SELECT svg(p) FROM geo ORDER BY id;
|
||||
SELECT svg(p, 'b') FROM geo ORDER BY id;
|
||||
SELECT svg((0., 0.), s) FROM geo ORDER BY id;
|
||||
SELECT svg(p, s) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Tuple(Float64, Float64)), s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([(0., 0.), (10, 0), (10, 10), (0, 10)], 'b', 1);
|
||||
INSERT INTO geo VALUES ([(1., 0.), (10, 0), (10, 10), (0, 10)], 'c', 2);
|
||||
INSERT INTO geo VALUES ([(2., 0.), (10, 0), (10, 10), (0, 10)], 'd', 3);
|
||||
SELECT svg(p) FROM geo ORDER BY id;
|
||||
SELECT svg(p, 'b') FROM geo ORDER BY id;
|
||||
SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)], s) FROM geo ORDER BY id;
|
||||
SELECT svg(p, s) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Array(Tuple(Float64, Float64))), s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'b', 1);
|
||||
INSERT INTO geo VALUES ([[(1., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'c', 2);
|
||||
INSERT INTO geo VALUES ([[(2., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'd', 3);
|
||||
SELECT svg(p) FROM geo ORDER BY id;
|
||||
SELECT svg(p, 'b') FROM geo ORDER BY id;
|
||||
SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], s) FROM geo ORDER BY id;
|
||||
SELECT svg(p, s) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Array(Array(Tuple(Float64, Float64)))), s String, id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'b', 1);
|
||||
INSERT INTO geo VALUES ([[[(1., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'c', 2);
|
||||
INSERT INTO geo VALUES ([[[(2., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'd', 3);
|
||||
SELECT svg(p) FROM geo ORDER BY id;
|
||||
SELECT svg(p, 'b') FROM geo ORDER BY id;
|
||||
SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], s) FROM geo ORDER BY id;
|
||||
SELECT svg(p, s) FROM geo ORDER BY id;
|
16
tests/queries/0_stateless/01300_wkt.reference
Normal file
16
tests/queries/0_stateless/01300_wkt.reference
Normal file
@ -0,0 +1,16 @@
|
||||
POINT(0 0)
|
||||
POLYGON((0 0,10 0,10 10,0 10))
|
||||
POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))
|
||||
MULTIPOLYGON(((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))
|
||||
POINT(0 0)
|
||||
POINT(1 0)
|
||||
POINT(2 0)
|
||||
POLYGON((0 0,10 0,10 10,0 10))
|
||||
POLYGON((1 0,10 0,10 10,0 10))
|
||||
POLYGON((2 0,10 0,10 10,0 10))
|
||||
POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))
|
||||
POLYGON((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4))
|
||||
POLYGON((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4))
|
||||
MULTIPOLYGON(((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))
|
||||
MULTIPOLYGON(((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))
|
||||
MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))
|
32
tests/queries/0_stateless/01300_wkt.sql
Normal file
32
tests/queries/0_stateless/01300_wkt.sql
Normal file
@ -0,0 +1,32 @@
|
||||
SELECT wkt((0., 0.));
|
||||
SELECT wkt([(0., 0.), (10., 0.), (10., 10.), (0., 10.)]);
|
||||
SELECT wkt([[(0., 0.), (10., 0.), (10., 10.), (0., 10.)], [(4., 4.), (5., 4.), (5., 5.), (4., 5.)]]);
|
||||
SELECT wkt([[[(0., 0.), (10., 0.), (10., 10.), (0., 10.)], [(4., 4.), (5., 4.), (5., 5.), (4., 5.)]], [[(-10., -10.), (-10., -9.), (-9., 10.)]]]);
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Tuple(Float64, Float64), id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ((0, 0), 1);
|
||||
INSERT INTO geo VALUES ((1, 0), 2);
|
||||
INSERT INTO geo VALUES ((2, 0), 3);
|
||||
SELECT wkt(p) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Tuple(Float64, Float64)), id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([(0, 0), (10, 0), (10, 10), (0, 10)], 1);
|
||||
INSERT INTO geo VALUES ([(1, 0), (10, 0), (10, 10), (0, 10)], 2);
|
||||
INSERT INTO geo VALUES ([(2, 0), (10, 0), (10, 10), (0, 10)], 3);
|
||||
SELECT wkt(p) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Array(Tuple(Float64, Float64))), id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 1);
|
||||
INSERT INTO geo VALUES ([[(1, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 2);
|
||||
INSERT INTO geo VALUES ([[(2, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 3);
|
||||
SELECT wkt(p) FROM geo ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS geo;
|
||||
CREATE TABLE geo (p Array(Array(Array(Tuple(Float64, Float64)))), id Int) engine=Memory();
|
||||
INSERT INTO geo VALUES ([[[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], [[(-10, -10), (-10, -9), (-9, 10)]]], 1);
|
||||
INSERT INTO geo VALUES ([[[(1, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], [[(-10, -10), (-10, -9), (-9, 10)]]], 2);
|
||||
INSERT INTO geo VALUES ([[[(2, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], [[(-10, -10), (-10, -9), (-9, 10)]]], 3);
|
||||
SELECT wkt(p) FROM geo ORDER BY id;
|
10
tests/queries/0_stateless/01301_polygons_within.reference
Normal file
10
tests/queries/0_stateless/01301_polygons_within.reference
Normal file
@ -0,0 +1,10 @@
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
-------- MultiPolygon with Polygon
|
||||
0
|
||||
-------- MultiPolygon with Polygon with Holes
|
||||
0
|
||||
-------- Polygon with Polygon with Holes
|
||||
0
|
15
tests/queries/0_stateless/01301_polygons_within.sql
Normal file
15
tests/queries/0_stateless/01301_polygons_within.sql
Normal file
File diff suppressed because one or more lines are too long
@ -0,0 +1,4 @@
|
||||
0
|
||||
1.2727922061357855
|
||||
0.3274195462417724
|
||||
0.3274195462417724
|
8
tests/queries/0_stateless/01302_polygons_distance.sql
Normal file
8
tests/queries/0_stateless/01302_polygons_distance.sql
Normal file
@ -0,0 +1,8 @@
|
||||
select polygonsDistanceCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
select polygonsDistanceCartesian([[[(0, 0), (0, 0.1), (0.1, 0.1), (0.1, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
select polygonsDistanceSpherical([[[(23.725750, 37.971536)]]], [[[(4.3826169, 50.8119483)]]]);
|
||||
|
||||
drop table if exists polygon_01302;
|
||||
create table polygon_01302 (x Array(Array(Array(Tuple(Float64, Float64)))), y Array(Array(Array(Tuple(Float64, Float64))))) engine=Memory();
|
||||
insert into polygon_01302 values ([[[(23.725750, 37.971536)]]], [[[(4.3826169, 50.8119483)]]]);
|
||||
select polygonsDistanceSpherical(x, y) from polygon_01302;
|
@ -0,0 +1,2 @@
|
||||
0
|
||||
1
|
2
tests/queries/0_stateless/01303_polygons_equals.sql
Normal file
2
tests/queries/0_stateless/01303_polygons_equals.sql
Normal file
@ -0,0 +1,2 @@
|
||||
select polygonsEqualsCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
||||
select polygonsEqualsCartesian([[[(1., 1.),(1., 4.),(4., 4.),(4., 1.)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]);
|
File diff suppressed because one or more lines are too long
10
tests/queries/0_stateless/01304_polygons_sym_difference.sql
Normal file
10
tests/queries/0_stateless/01304_polygons_sym_difference.sql
Normal file
File diff suppressed because one or more lines are too long
8
tests/queries/0_stateless/01305_polygons_union.reference
Normal file
8
tests/queries/0_stateless/01305_polygons_union.reference
Normal file
@ -0,0 +1,8 @@
|
||||
[[[(1,2.9),(1,4),(4,4),(4,1),(2.9,1),(3,0),(0,0),(0,3),(1,2.9)]]]
|
||||
[[[(4.3666052904432435,50.84337386140151),(4.366227,50.840809),(4.344961,50.833264),(4.338074,50.848677),(4.346693,50.858306),(4.3526804582393535,50.856658100365976),(4.3613577,50.8651821),(4.3613148,50.8651279),(4.3904543,50.8564867),(4.3830299,50.8428851),(4.3666052904432435,50.84337386140151)]]]
|
||||
-------- MultiPolygon with Polygon
|
||||
MULTIPOLYGON(((35.5408 58.9593,37.2817 59.9768,38.7325 59.9465,36.9725 59.0149,37.3119 59.0258,37.8553 58.9075,39.7299 59.9314,44.4751 59.81,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,39.1456 52.7573,38.0214 52.8989,37.1608 52.2393,35.4682 52.2022,36.5022 53.0008,35.3776 53.0462,35.3645 53.076,34.2895 52.2208,32.5969 52.2208,33.4048 52.8423,33.1712 52.8276,32.5275 53.1741,31.2368 52.1652,29.7861 52.1466,32.2523 53.964,31.8748 54.1736,29.3931 52.2763,29.4536 59.7796,30.5719 59.9919,30.4812 58.8542,32.3249 59.9465,33.6548 59.9465,30.179 57.9196,30.179 56.9764,32.2175 58.3664,32.2342 58.4928,32.5691 58.5924,34.8637 59.9768,36.2843 59.9616,34.0496 58.6717,34.9952 58.6226,35.3712 58.8556,34.6522 58.9167,35.5408 58.9593),(36.4989 58.7512,36.1498 58.553,36.3447 58.5402,36.0877 58.5174,35.4314 58.1349,36.403 58.0507,36.5949 58.1673,36.0123 58.2869,37.191 58.6819,36.4989 58.7512),(34.4816 56.8232,34.8098 57.0409,33.5602 56.9781,33.3418 56.8364,33.8361 56.6953,34.1885 56.6259,34.3867 56.7596,34.229 56.7948,34.4816 56.8232),(35.9179 57.7512,35.7402 57.7909,36.0848 57.855,36.3932 58.0447,35.1134 57.9454,34.6332 57.6538,35.613 57.5595,35.9179 57.7512),(36.8709 53.2765,37.135 53.4711,36.9794 53.5878,37.3119 53.9273,37.0035 54.2999,36.6985 54.0791,36.919 53.8561,36.3552 53.8269,36.1528 53.6763,36.8709 53.2765),(38.1601 55.1091,38.3093 55.1546,38.2609 55.1775,38.1601 55.1091),(38.1688 56.0758,38.4339 56.2361,37.5054 56.5484,37.2281 56.3799,38.1688 56.0758),(38.1319 56.0534,36.647 55.9411,37.6238 55.7402,38.1319 56.0534),(37.2824 55.5258,36.8283 55.4471,37.06 55.3843,37.2824 55.5258),(36.151 54.791,36.0123 54.7554,36.0472 54.7217,36.151 54.791),(34.9611 53.9765,34.894 54.1226,35.6193 54.4929,34.9706 54.9262,34.7231 54.7576,35.0753 54.5981,34.1081 54.1757,34.7279 53.8116,34.9611 53.9765),(38.2312 56.9795,37.565 56.5843,38.9742 56.8774,38.4915 57.1308,38.2699 57.0021,38.3093 56.9929,38.2312 56.9795),(36.5334 56.6753,36.375 56.6455,36.4446 56.6242,36.5334 56.6753),(36.1999 57.0022,36.9794 57.0751,36.4587 57.1544,36.1999 57.0022),(34.6028 58.3749,33.6245 58.271,34.3593 58.2189,34.6028 58.3749),(33.7581 57.8255,33.2316 57.7748,33.6325 57.7419,33.7581 57.8255),(31.6069 56.3194,31.7506 56.8609,31.6514 57.1258,30.3301 56.1942,30.2394 55.2753,31.6069 56.3194),(34.2274 57.4023,34.0208 57.2724,35.0338 57.1875,35.4682 57.4674,34.2274 57.4023),(31.7782 55.7778,30.2092 54.6331,30.2394 53.6774,31.7439 54.8677,31.8413 54.9989,32.204 55.5156,31.7782 55.7778),(33.7222 56.3063,32.8387 56.3117,33.5244 56.1686,33.7222 56.3063),(33.1204 55.8832,32.748 55.9072,32.9547 55.7645,33.1204 55.8832),(35.2275 55.0993,36.4354 55.3441,35.7505 55.4454,35.2275 55.0993),(35.9817 55.5958,36.5563 55.6352,36.193 55.7319,35.9817 55.5958),(35.0954 55.822,35.3188 55.9582,34.7331 56.1049,34.4996 55.9565,35.0954 55.822),(34.9721 55.7463,34.2598 55.8023,33.6125 55.3778,34.3709 55.3709,34.9721 55.7463),(35.6571 56.1619,36.0233 56.3789,35.4083 56.5254,35.2273 56.414,35.71 56.3117,35.0485 56.303,34.744 56.1118,35.6571 56.1619),(40.2143 54.467,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,39.9942 53.358,43.0243 55.3269,43.0243 56.2614,40.2143 54.467),(38.5511 53.2922,38.4609 53.226,39.2704 52.8471,39.9877 53.3534,38.5511 53.2922),(40.5716 55.8007,43.0243 57.2554,43.0243 58.0797,40.4543 56.5923,40.4855 56.4957,40.2529 56.4682,39.8102 56.1914,39.8205 56.0763,40.425 56.1942,40.5716 55.8007),(40.5504 55.7875,39.7601 55.7544,39.8151 55.3187,40.5504 55.7875),(39.7863 57.025,42.5105 58.477,41.6944 58.8542,40.1389 58.048,40.2437 58.0478,40.3343 57.4673,39.7299 57.4673,39.7863 57.025),(38.0744 57.5312,38.3737 57.6908,38.3395 57.7103,38.8533 58.0638,38.432 58.2584,38.0535 58.0542,38.3395 57.9356,37.4328 57.7103,38.0744 57.5312),(37.9669 57.4734,37.1608 57.2554,37.4489 57.1909,37.9669 57.4734),(40.4136 58.7241,41.2108 59.1035,40.6366 59.3817,39.8163 58.9766,40.4552 58.9011,40.4136 58.7241),(39.7184 58.3823,39.6392 58.3821,39.6392 58.3427,39.7184 58.3823),(38.7465 58.4255,39.5485 58.7133,39.4085 58.7696,38.7465 58.4255)))
|
||||
-------- MultiPolygon with Polygon with Holes
|
||||
MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1455,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(33.1079 56.9523,33.1392 56.8934,33.7182 56.7292,35.1489 56.5859,34.229 56.7948,36.9794 57.0751,35.7705 57.2554,37.0097 57.4998,35.7402 57.7909,37.1608 58.0478,36.0123 58.2869,37.191 58.6819,34.6522 58.9167,37.2327 59.0233,37.1118 59.6677,35.1343 59.8448,31.9702 58.9727,32.25 58.4976,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,36.3447 58.5402,33.6245 58.271,36.4354 58.0478,33.2316 57.7748,36.1936 57.4998,33.1712 57.337,36.0727 57.0915,33.1079 56.9523),(37.0604 52.9744,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,42.8247 56.5837,40.4412 56.1511,40.5761 55.7884,39.7601 55.7544,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,40.0019 53.354,38.3395 53.2817,39.5787 52.6996,37.8559 52.9188,37.4471 53.2343,37.2165 53.0798,37.4328 52.9552,37.0604 52.9744),(31.627 54.7093,29.5972 55.5037,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,30.1245 53.1731,32.4808 53.1989,32.0831 53.408,32.476 53.8383,31.4182 54.4227,31.627 54.7093),(34.7731 53.3243,34.7731 53.1793,35.0903 53.1731,34.7731 53.3243),(36.9508 55.414,37.7653 55.1891,36.8822 54.975,37.0572 54.7635,38.3093 55.1546,37.7955 55.3956,38.4907 55.5327,38.3184 55.7179,38.0262 55.6546,38.0373 55.6523,37.9482 55.6376,36.9508 55.414),(38.3092 56.9929,38.5798 57.0849,38.2186 57.2717,38.7325 57.4835,38.3395 57.7103,38.8533 58.0638,38.3698 58.2869,39.5485 58.7133,38.8838 58.9777,38.0944 58.8545,38.5813 58.7446,37.4026 58.3187,38.3395 57.9356,37.4328 57.7103,38.128 57.516,37.1608 57.2554,38.3092 56.9929),(38.309 56.9928,36.375 56.6455,36.8799 56.4895,38.309 56.9928),(40.3237 57.5365,42.6929 58.0314,40.8911 59.2659,39.2792 59.0373,40.4552 58.9011,40.3343 58.3821,39.6392 58.3821,39.6392 58.0478,40.2437 58.0478,40.3237 57.5365),(40.0149 57.4677,39.7299 57.4673,39.7379 57.4051,40.0149 57.4677)))
|
||||
-------- Polygon with Polygon with Holes
|
||||
MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1455,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(32.6512 57.792,32.9378 57.2699,36.7912 59.6986,35.9475 59.7758,32.6512 57.792),(33.2446 56.7729,34.2635 56.6767,37.6322 58.7797,37.2876 58.7226,37.2102 59.1452,33.2446 56.7729),(36.1815 56.4715,41.168 59.0834,40.9299 59.2404,40.8804 59.2644,40.2079 59.1718,35.4536 56.5531,36.1815 56.4715),(30.7705 55.0525,30.2092 54.6331,30.2394 53.6774,31.5682 54.7333,30.7705 55.0525),(33.8733 53.1922,34.3351 53.53,33.5144 53.9057,32.5603 53.1989,33.8733 53.1922),(31.1968 52.1649,29.7861 52.1466,30.5785 52.7531,30.3098 53.0028,29.3931 52.2763,29.4171 55.606,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,31.1968 52.1649),(31.1682 53.1903,32.6907 54.2663,32.2591 54.4483,30.5408 53.1811,31.1682 53.1903),(39.4328 55.9511,37.2766 54.4948,37.7431 53.9104,41.4519 56.3413,39.4328 55.9511),(40.9691 57.677,42.2498 58.3455,41.5887 58.8012,38.1759 56.9472,39.0894 57.2553,40.9691 57.677),(37.1934 55.4694,36.5845 55.3291,36.7219 55.1665,37.1934 55.4694),(32.2964 58.4175,34.2247 59.6064,31.9702 58.9727,32.2964 58.4175),(35.9681 52.2157,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,44.4212 55.8594,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,43.0243 55.3269,43.0243 56.2614,37.1608 52.2393,35.9681 52.2157)))
|
15
tests/queries/0_stateless/01305_polygons_union.sql
Normal file
15
tests/queries/0_stateless/01305_polygons_union.sql
Normal file
File diff suppressed because one or more lines are too long
@ -0,0 +1,10 @@
|
||||
[[[(1,2.9),(2,2.6),(2.6,2),(2.9,1),(1,1),(1,2.9)]]]
|
||||
[]
|
||||
[]
|
||||
[[[(4.3666052904432435,50.84337386140151),(4.3602419,50.8435626),(4.349556,50.8535879),(4.3526804582393535,50.856658100365976),(4.367945,50.852455),(4.3666052904432435,50.84337386140151)]]]
|
||||
-------- MultiPolygon with Polygon
|
||||
MULTIPOLYGON(((35.5408 58.9593,36.9725 59.0149,36.4989 58.7512,35.3712 58.8556,35.5408 58.9593)),((34.4816 56.8232,36.1999 57.0022,35.4083 56.5254,34.3867 56.7596,34.4816 56.8232)),((35.9179 57.7512,36.0848 57.855,37.1608 58.0478,36.5949 58.1673,37.8553 58.9075,38.5813 58.7446,37.4026 58.3187,38.0535 58.0542,36.4587 57.1544,35.7705 57.2554,37.0097 57.4998,35.9179 57.7512)),((36.8709 53.2765,37.4328 52.9552,36.5022 53.0008,36.8709 53.2765)),((36.1528 53.6763,35.3645 53.076,34.9611 53.9765,36.0472 54.7217,36.6985 54.0791,36.3552 53.8269,35.9216 53.8026,36.1528 53.6763)),((37.0035 54.2999,36.7074 54.6506,38.1601 55.1091,37.0035 54.2999)),((38.1688 56.0758,38.2186 56.0594,38.1319 56.0534,38.1688 56.0758)),((37.6238 55.7402,38.0373 55.6523,37.2824 55.5258,37.6238 55.7402)),((37.06 55.3843,37.7653 55.1891,36.151 54.791,37.06 55.3843)),((38.2312 56.9795,36.5334 56.6753,37.4489 57.1909,38.2699 57.0021,38.2312 56.9795)),((37.2281 56.3799,36.193 55.7319,35.3188 55.9582,35.6571 56.1619,36.7074 56.211,36.0233 56.3789,36.4446 56.6242,37.2281 56.3799)),((34.9952 58.6226,36.1498 58.553,36.0877 58.5174,34.6028 58.3749,34.9952 58.6226)),((34.3593 58.2189,35.4314 58.1349,35.1134 57.9454,33.7581 57.8255,34.3593 58.2189)),((33.6325 57.7419,34.6332 57.6538,34.2274 57.4023,33.1712 57.337,34.0208 57.2724,33.5602 56.9781,32.9596 56.9434,33.3418 56.8364,31.7782 55.7778,31.5088 55.9411,31.6069 56.3194,33.6325 57.7419)),((36.403 58.0507,36.4354 58.0478,36.3932 58.0447,36.403 58.0507)),((35.613 57.5595,36.1936 57.4998,35.4682 57.4674,35.613 57.5595)),((35.0338 57.1875,36.0727 57.0915,34.8098 57.0409,35.0338 57.1875)),((34.1885 56.6259,35.2273 56.414,35.0485 56.303,34.5917 56.2949,33.7222 56.3063,34.1885 56.6259)),((33.5244 56.1686,34.4996 55.9565,34.2598 55.8023,33.1204 55.8832,33.5244 56.1686)),((32.9547 55.7645,33.5036 55.3785,33.6125 55.3778,31.8748 54.1736,31.4182 54.4227,31.7439 54.8677,32.9547 55.7645)),((34.7279 53.8116,34.7731 53.7847,34.7731 52.9188,33.4048 52.8423,34.7279 53.8116)),((34.7231 54.7576,32.5275 53.1741,32.0831 53.408,32.476 53.8383,32.2523 53.964,34.3709 55.3709,35.0149 55.3613,34.2593 54.9642,34.7231 54.7576)),((34.9706 54.9262,34.8335 55.0162,35.2275 55.0993,34.9706 54.9262)),((35.7505 55.4454,35.1358 55.5327,35.9817 55.5958,35.7505 55.4454)),((35.0954 55.822,35.6798 55.6863,34.9721 55.7463,35.0954 55.822)),((34.7331 56.1049,34.7126 56.11,34.744 56.1118,34.7331 56.1049)),((40.2143 54.467,38.5511 53.2922,38.3395 53.2817,38.4609 53.226,38.0214 52.8989,37.8559 52.9188,37.135 53.4711,39.8151 55.3187,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.2143 54.467)),((40.5716 55.8007,40.5761 55.7884,40.5504 55.7875,40.5716 55.8007)),((40.4543 56.5923,40.2529 56.4682,39.7903 56.4121,39.8102 56.1914,38.2609 55.1775,37.7955 55.3956,38.4907 55.5327,38.1884 55.8564,38.944 56.0594,38.4339 56.2361,39.7863 57.025,39.7903 56.9929,40.3343 56.9599,40.4543 56.5923)),((40.1389 58.048,38.4915 57.1308,38.2186 57.2717,38.7325 57.4835,38.3737 57.6908,39.6392 58.3427,39.6392 58.0478,40.1389 58.048)),((37.5054 56.5484,37.463 56.5623,37.565 56.5843,37.5054 56.5484)),((38.0744 57.5312,38.128 57.516,37.9669 57.4734,38.0744 57.5312)),((40.4136 58.7241,40.3343 58.3821,39.7184 58.3823,40.4136 58.7241)),((39.8163 58.9766,39.4085 58.7696,38.5209 59.119,39.8163 58.9766)),((38.432 58.2584,38.3698 58.2869,38.7465 58.4255,38.432 58.2584)),((32.2175 58.3664,32.5691 58.5924,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,34.0496 58.6717,31.6514 57.1258,31.5088 57.4998,32.1738 58.0318,32.2175 58.3664)),((39.9942 53.358,40.0019 53.354,39.9877 53.3534,39.9942 53.358)),((39.2704 52.8471,39.5787 52.6996,39.1456 52.7573,39.2704 52.8471)))
|
||||
-------- MultiPolygon with Polygon with Holes
|
||||
MULTIPOLYGON(((33.1079 56.9523,32.9596 56.9434,33.1392 56.8934,33.2007 56.7768,33.7182 56.7292,33.8361 56.6953,35.71 56.3117,34.5917 56.2949,32.8387 56.3117,35.6798 55.6863,32.748 55.9072,33.5036 55.3785,35.0149 55.3613,34.2593 54.9642,35.0753 54.5981,34.1081 54.1757,34.7731 53.7847,34.7731 53.3243,33.1128 54.0852,31.627 54.7093,31.8413 54.9989,32.204 55.5156,31.5088 55.9411,31.7506 56.8609,31.5088 57.4998,32.1738 58.0318,32.2342 58.4928,32.25 58.4976,33.1079 56.9523)),((35.1489 56.5859,36.6724 56.4139,36.8799 56.4895,38.2186 56.0594,36.647 55.9411,38.0262 55.6546,37.9482 55.6376,36.8283 55.4471,36.9508 55.414,36.5845 55.3291,36.8822 54.975,36.0123 54.7554,36.919 53.8561,35.9216 53.8026,37.2165 53.0798,37.0604 52.9744,35.3776 53.0462,34.894 54.1226,35.6193 54.4929,34.8335 55.0162,36.4354 55.3441,35.1358 55.5327,36.5563 55.6352,34.7126 56.11,36.7074 56.211,35.1489 56.5859)),((37.2327 59.0233,37.3119 59.0258,38.0944 58.8545,37.2876 58.7226,37.2327 59.0233)),((37.4471 53.2343,36.9794 53.5878,37.3119 53.9273,36.7074 54.6506,37.0572 54.7635,37.9907 53.5925,37.4471 53.2343)),((34.7731 53.1793,34.7731 52.9188,33.1712 52.8276,32.4808 53.1989,34.7731 53.1793)),((40.4412 56.1511,38.3184 55.7179,38.1884 55.8564,38.944 56.0594,37.463 56.5623,38.9742 56.8774,38.5798 57.0849,39.0894 57.2553,39.7379 57.4051,39.7903 56.9929,40.3343 56.9599,40.4855 56.4957,39.7903 56.4121,39.8205 56.0763,40.425 56.1942,40.4412 56.1511)),((38.3092 56.9929,38.3093 56.9929,38.309 56.9928,38.3092 56.9929)),((40.3237 57.5365,40.3343 57.4673,40.0149 57.4677,40.3237 57.5365)),((39.2792 59.0373,38.8838 58.9777,38.5209 59.119,39.2792 59.0373)))
|
||||
-------- Polygon with Polygon with Holes
|
||||
MULTIPOLYGON(((32.6512 57.792,30.3301 56.1942,30.2394 55.2753,32.9378 57.2699,33.2007 56.7768,33.2446 56.7729,30.7705 55.0525,29.5972 55.5037,29.4171 55.606,29.4536 59.7796,30.5719 59.9919,30.4812 58.8542,32.3249 59.9465,33.6548 59.9465,30.179 57.9196,30.179 56.9764,32.2964 58.4175,32.6512 57.792)),((35.9475 59.7758,35.1343 59.8448,34.2247 59.6064,34.8637 59.9768,36.2843 59.9616,35.9475 59.7758)),((36.7912 59.6986,37.2817 59.9768,38.7325 59.9465,37.2102 59.1452,37.1118 59.6677,36.7912 59.6986)),((34.2635 56.6767,35.4536 56.5531,32.2591 54.4483,31.5682 54.7333,34.2635 56.6767)),((36.1815 56.4715,36.6724 56.4139,38.1759 56.9472,33.5144 53.9057,33.1128 54.0852,32.6907 54.2663,36.1815 56.4715)),((33.8733 53.1922,35.0903 53.1731,34.3351 53.53,36.7219 55.1665,37.2766 54.4948,34.2895 52.2208,32.5969 52.2208,33.8733 53.1922)),((31.1968 52.1649,30.5785 52.7531,31.1682 53.1903,32.5603 53.1989,31.2368 52.1652,31.1968 52.1649)),((30.3098 53.0028,30.1245 53.1731,30.5408 53.1811,30.3098 53.0028)),((37.6322 58.7797,39.7299 59.9314,44.4751 59.81,44.4212 55.8594,42.8247 56.5837,41.4519 56.3413,43.0243 57.2554,43.0243 58.0797,39.4328 55.9511,37.1934 55.4694,40.9691 57.677,42.6929 58.0314,42.2498 58.3455,42.5105 58.477,41.6944 58.8542,41.5887 58.8012,41.168 59.0834,41.2108 59.1035,40.9299 59.2404,40.8911 59.2659,40.8804 59.2644,40.6366 59.3817,40.2079 59.1718,37.6322 58.7797)),((35.9681 52.2157,35.4682 52.2022,37.7431 53.9104,37.9907 53.5925,35.9681 52.2157)))
|
14
tests/queries/0_stateless/01306_polygons_intersection.sql
Normal file
14
tests/queries/0_stateless/01306_polygons_intersection.sql
Normal file
File diff suppressed because one or more lines are too long
@ -0,0 +1 @@
|
||||
20
|
1
tests/queries/0_stateless/01307_polygon_perimeter.sql
Normal file
1
tests/queries/0_stateless/01307_polygon_perimeter.sql
Normal file
@ -0,0 +1 @@
|
||||
select polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (0., 0.)]]]);
|
2
tests/queries/0_stateless/01308_polygon_area.reference
Normal file
2
tests/queries/0_stateless/01308_polygon_area.reference
Normal file
@ -0,0 +1,2 @@
|
||||
25
|
||||
9.387703638370358e-8
|
3
tests/queries/0_stateless/01308_polygon_area.sql
Normal file
3
tests/queries/0_stateless/01308_polygon_area.sql
Normal file
@ -0,0 +1,3 @@
|
||||
select polygonAreaCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]);
|
||||
select polygonAreaSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]);
|
||||
SELECT polygonAreaCartesian([]); -- { serverError 36 }
|
@ -1 +1 @@
|
||||
SELECT pointInPolygon((0, 0), [[(0, 0), (10, 10), (256, -9223372036854775808)]]) FORMAT Null;
|
||||
SELECT pointInPolygon((0, 0), [[(0, 0), (10, 10), (256, -9223372036854775808)]]) FORMAT Null ;-- { serverError 36 }
|
||||
|
104
tests/queries/0_stateless/01720_country_intersection.reference
Normal file
104
tests/queries/0_stateless/01720_country_intersection.reference
Normal file
File diff suppressed because one or more lines are too long
18
tests/queries/0_stateless/01720_country_intersection.sh
Executable file
18
tests/queries/0_stateless/01720_country_intersection.sh
Executable file
@ -0,0 +1,18 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_polygons;"
|
||||
${CLICKHOUSE_CLIENT} -q "create table country_polygons(name String, p Array(Array(Tuple(Float64, Float64)))) engine=MergeTree() order by tuple();"
|
||||
cat ${CURDIR}/country_polygons.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_polygons format TSV"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT c, d, polygonsIntersectionSpherical(a, b) FROM (SELECT first.p AS a, second.p AS b, first.name AS c, second.name AS d FROM country_polygons AS first CROSS JOIN country_polygons AS second LIMIT 100) format TSV"
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_polygons;"
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;"
|
||||
${CLICKHOUSE_CLIENT} -q "create table country_rings(name String, p Array(Tuple(Float64, Float64))) engine=MergeTree() order by tuple();"
|
||||
cat ${CURDIR}/country_rings.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_rings format TSV"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT c, d, polygonsIntersectionSpherical(a, b) FROM (SELECT first.p AS a, second.p AS b, first.name AS c, second.name AS d FROM country_rings AS first CROSS JOIN country_rings AS second LIMIT 100) format TSV"
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;"
|
@ -0,0 +1,214 @@
|
||||
Dhekelia Sovereign Base Area 0.0186259930051051
|
||||
Kyrgyzstan 0.5868323961091907
|
||||
-------------------------------------
|
||||
Dhekelia Sovereign Base Area 0.000003139488070896512
|
||||
Kyrgyzstan 0.004895645023822883
|
||||
-------------------------------------
|
||||
Aruba 0.011249330810410983
|
||||
Afghanistan 0.8199216326776404
|
||||
Albania 0.17108622597702605
|
||||
Andorra 0.015145740647213184
|
||||
Ashmore and Cartier Islands 0.001111472909012953
|
||||
Austria 0.3258464621357028
|
||||
Burundi 0.1409500621452211
|
||||
Belgium 0.1794463601873955
|
||||
Benin 0.31426073515874664
|
||||
Burkina Faso 0.5144381682226761
|
||||
Bulgaria 0.3083164214454252
|
||||
Bahrain 0.02137170357214413
|
||||
Bosnia and Herzegovina 0.20611959113245232
|
||||
Bajo Nuevo Bank (Petrel Is.) 0.0001254597070361587
|
||||
Saint Barthelemy 0.0032990108720812672
|
||||
Belarus 0.42899119772830474
|
||||
Bolivia 0.9279328001326348
|
||||
Barbados 0.014116142490651021
|
||||
Bhutan 0.1601735058766338
|
||||
Botswana 0.5896697538755427
|
||||
Central African Republic 0.7760222837198817
|
||||
Switzerland 0.2318851512510408
|
||||
Clipperton Island 0.0014072924221565273
|
||||
Cameroon 0.8001045813665599
|
||||
Republic of Congo 0.6904316055863188
|
||||
Coral Sea Islands 0.00011634674137689659
|
||||
Curaçao 0.02078862020307983
|
||||
Czech Republic 0.2708588915805718
|
||||
Djibouti 0.12937731543684822
|
||||
Dominica 0.020094439807419574
|
||||
Algeria 1.1549683948032776
|
||||
Ethiopia 0.8210654364815099
|
||||
Georgia 0.26823008017781313
|
||||
Ghana 0.4056578143818251
|
||||
Gibraltar 0.0014059440610631154
|
||||
Guinea 0.6350853755877334
|
||||
Gambia 0.19279774895359095
|
||||
Guatemala 0.3030953561509038
|
||||
Guam 0.020321390076536976
|
||||
Heard Island and McDonald Islands 0.017334896920453105
|
||||
Hungary 0.2617732480910806
|
||||
Isle of Man 0.01875803631141408
|
||||
Iraq 0.5469861219502402
|
||||
Israel 0.19353851895699914
|
||||
Jamaica 0.10055860979159512
|
||||
Jersey 0.008427337812134537
|
||||
Jordan 0.2642243503964102
|
||||
Baykonur Cosmodrome 0.04482995477542441
|
||||
Siachen Glacier 0.03872116827341272
|
||||
Kosovo 0.08773172991408161
|
||||
Laos 0.6899867972760174
|
||||
Lebanon 0.09676977254650951
|
||||
Liberia 0.2961649538030388
|
||||
Libya 0.9538430912224716
|
||||
Saint Lucia 0.016786201647759867
|
||||
Liechtenstein 0.009288582116863231
|
||||
Lesotho 0.12315874900320756
|
||||
Luxembourg 0.04125996057810259
|
||||
Latvia 0.24488610945731157
|
||||
Saint Martin 0.006547834154217771
|
||||
Morocco 0.8817924249630141
|
||||
Monaco 0.0026049777439637527
|
||||
Moldova 0.20765701819586885
|
||||
Macedonia 0.1128831074330059
|
||||
Mali 1.1385970015559317
|
||||
Montenegro 0.11756794062084858
|
||||
Mongolia 1.142306166871007
|
||||
Montserrat 0.006620100691409788
|
||||
Namibia 0.843464957679987
|
||||
Niger 0.8780744302377772
|
||||
Norfolk Island 0.004912027225339993
|
||||
Niue 0.009881892958363517
|
||||
Nepal 0.4076113675280835
|
||||
Nauru 0.0031205159769295255
|
||||
Poland 0.48922069488271314
|
||||
Paraguay 0.5475256537493991
|
||||
Qatar 0.09362771431858698
|
||||
Romania 0.44095021664473105
|
||||
Rwanda 0.1293663890297039
|
||||
Western Sahara 0.4691920993279596
|
||||
Scarborough Reef 0.00019842225207367386
|
||||
South Sudan 0.7584190842556537
|
||||
Senegal 0.5883247226863264
|
||||
Serranilla Bank 0.0002389083935906293
|
||||
Singapore 0.015233384733369614
|
||||
San Marino 0.004596873449598911
|
||||
Somaliland 0.3096791489207226
|
||||
Somalia 0.6879915318072617
|
||||
Republic of Serbia 0.29677234233404165
|
||||
Suriname 0.32255243342976203
|
||||
Slovakia 0.19843599488831584
|
||||
Slovenia 0.14713148471782736
|
||||
Swaziland 0.08434161089555517
|
||||
Sint Maarten 0.0037955305365309296
|
||||
Syria 0.35675522352394456
|
||||
Chad 0.9102578296637189
|
||||
Togo 0.2600585482954555
|
||||
Uganda 0.38301730108810556
|
||||
Uruguay 0.3083564407046887
|
||||
Vatican 0.00006702452496391445
|
||||
Akrotiri Sovereign Base Area 0.013376747415600219
|
||||
Zambia 0.8807923488623808
|
||||
Zimbabwe 0.4553903789902945
|
||||
-------------------------------------
|
||||
Aruba 0.0000041986375296795025
|
||||
Afghanistan 0.015826481758320493
|
||||
Albania 0.0006971811189621746
|
||||
Andorra 0.00001112355564980348
|
||||
Ashmore and Cartier Islands 6.66668338977609e-8
|
||||
Austria 0.0020634744883290235
|
||||
Burundi 0.000669169243101558
|
||||
Belgium 0.0007529367590741593
|
||||
Benin 0.00287239734953164
|
||||
Burkina Faso 0.006746218025419332
|
||||
Bulgaria 0.0027733372191197786
|
||||
Bahrain 0.00001443842547561405
|
||||
Bosnia and Herzegovina 0.0012742491201009779
|
||||
Bajo Nuevo Bank (Petrel Is.) 8.864825701897049e-10
|
||||
Saint Barthelemy 6.036607210116289e-7
|
||||
Belarus 0.005090738074359067
|
||||
Bolivia 0.026865324735758436
|
||||
Barbados 0.0000109856680212211
|
||||
Bhutan 0.0009961026696220909
|
||||
Botswana 0.01430200501713062
|
||||
Central African Republic 0.015290667187215962
|
||||
Switzerland 0.0010181463734151514
|
||||
Clipperton Island 1.2373029819547803e-7
|
||||
Cameroon 0.011488908713113137
|
||||
Republic of Congo 0.008534881807187833
|
||||
Coral Sea Islands 5.121674593493771e-10
|
||||
Curaçao 0.000011457378136273848
|
||||
Czech Republic 0.0019339153549488386
|
||||
Djibouti 0.000540370985929321
|
||||
Dominica 0.000018056168258583246
|
||||
Algeria 0.05696762706232162
|
||||
Ethiopia 0.02789047634482515
|
||||
Georgia 0.0017113229913929072
|
||||
Ghana 0.0059048504621945965
|
||||
Gibraltar 9.095456688875715e-8
|
||||
Guinea 0.006043151808047173
|
||||
Gambia 0.0002596816395280707
|
||||
Guatemala 0.0026901925526205263
|
||||
Guam 0.000013952443476670549
|
||||
Heard Island and McDonald Islands 0.000009688375334192321
|
||||
Hungary 0.0022899094702118978
|
||||
Isle of Man 0.00001410012284549863
|
||||
Iraq 0.010780689598789812
|
||||
Israel 0.0005400181032289429
|
||||
Jamaica 0.00027268062650994383
|
||||
Jersey 0.0000029236161155167853
|
||||
Jordan 0.002191215069390572
|
||||
Baykonur Cosmodrome 0.00015978303781425133
|
||||
Siachen Glacier 0.0000513879615262916
|
||||
Kosovo 0.0002684178325412152
|
||||
Laos 0.005637555524983489
|
||||
Lebanon 0.0002464436461544738
|
||||
Liberia 0.002357973807538481
|
||||
Libya 0.040072512808839354
|
||||
Saint Lucia 0.000014963842166249258
|
||||
Liechtenstein 0.0000033722024322722466
|
||||
Lesotho 0.0007426290112070925
|
||||
Luxembourg 0.00006405006804909529
|
||||
Latvia 0.00158313668683266
|
||||
Saint Martin 0.00000168759530251474
|
||||
Morocco 0.014595589778269167
|
||||
Monaco 4.6325700981005285e-7
|
||||
Moldova 0.0008158639460823913
|
||||
Macedonia 0.0006245180554490506
|
||||
Mali 0.03096381132470007
|
||||
Montenegro 0.00033762445623993013
|
||||
Mongolia 0.038446609480001344
|
||||
Montserrat 0.0000024620326175206004
|
||||
Namibia 0.020320978539029165
|
||||
Niger 0.02919849042641136
|
||||
Norfolk Island 0.0000010150641235563077
|
||||
Niue 0.000005450796200539049
|
||||
Nepal 0.003629565673884544
|
||||
Nauru 7.119067469952887e-7
|
||||
Poland 0.0076921097527402876
|
||||
Paraguay 0.009875843128670564
|
||||
Qatar 0.0002752610716836153
|
||||
Romania 0.005809479702080411
|
||||
Rwanda 0.0006262235765421803
|
||||
Western Sahara 0.0022344529652030694
|
||||
Scarborough Reef 2.4176335726807567e-9
|
||||
South Sudan 0.015509656314462458
|
||||
Senegal 0.00485201810074574
|
||||
Serranilla Bank 2.6035559945372385e-9
|
||||
Singapore 0.000012633505579848072
|
||||
San Marino 0.0000014830814619737624
|
||||
Somaliland 0.0041412916217828406
|
||||
Somalia 0.011674654119996183
|
||||
Republic of Serbia 0.001907268740192651
|
||||
Suriname 0.0035911641359236534
|
||||
Slovakia 0.0011901587428922095
|
||||
Slovenia 0.0004995546076509384
|
||||
Swaziland 0.00042234053226485263
|
||||
Sint Maarten 5.772865969377286e-7
|
||||
Syria 0.004581243750467663
|
||||
Chad 0.0313064894302088
|
||||
Togo 0.0014067991034602252
|
||||
Uganda 0.005985159048654327
|
||||
Uruguay 0.0043716082436750115
|
||||
Vatican 3.002600504657064e-10
|
||||
Akrotiri Sovereign Base Area 0.0000024314362587592923
|
||||
Zambia 0.018594119224502336
|
||||
Zimbabwe 0.009621356779606268
|
||||
-------------------------------------
|
25
tests/queries/0_stateless/01720_country_perimeter_and_area.sh
Executable file
25
tests/queries/0_stateless/01720_country_perimeter_and_area.sh
Executable file
@ -0,0 +1,25 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_polygons;"
|
||||
${CLICKHOUSE_CLIENT} -q "create table country_polygons(name String, p Array(Array(Tuple(Float64, Float64)))) engine=MergeTree() order by tuple();"
|
||||
cat ${CURDIR}/country_polygons.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_polygons format TSV"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT name, polygonPerimeterSpherical(p) from country_polygons"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT name, polygonAreaSpherical(p) from country_polygons"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;"
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "create table country_rings(name String, p Array(Tuple(Float64, Float64))) engine=MergeTree() order by tuple();"
|
||||
cat ${CURDIR}/country_rings.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_rings format TSV"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT name, polygonPerimeterSpherical(p) from country_rings"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT name, polygonAreaSpherical(p) from country_rings"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
|
||||
${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;"
|
@ -218,3 +218,6 @@
|
||||
01682_cache_dictionary_complex_key
|
||||
01684_ssd_cache_dictionary_simple_key
|
||||
01685_ssd_cache_dictionary_complex_key
|
||||
01304_polygons_sym_difference
|
||||
01305_polygons_union
|
||||
01306_polygons_intersection
|
||||
|
2
tests/queries/0_stateless/country_polygons.tsv
Normal file
2
tests/queries/0_stateless/country_polygons.tsv
Normal file
File diff suppressed because one or more lines are too long
103
tests/queries/0_stateless/country_rings.tsv
Normal file
103
tests/queries/0_stateless/country_rings.tsv
Normal file
File diff suppressed because one or more lines are too long
@ -1 +1,5 @@
|
||||
# We have no suppressions!
|
||||
# https://github.com/llvm-mirror/compiler-rt/blob/master/lib/ubsan/ubsan_checks.inc
|
||||
|
||||
# Some value is outside the range of representable values of type 'long' on user-provided data inside boost::geometry - ignore.
|
||||
src:*/Functions/pointInPolygon.cpp
|
||||
src:*/contrib/boost/boost/geometry/*
|
||||
|
Loading…
Reference in New Issue
Block a user