Initial commit

This commit is contained in:
Andr0901 2021-04-29 01:16:45 +03:00 committed by Nikita Mikhaylov
parent fc783be800
commit 12219391d6
10 changed files with 683 additions and 5 deletions

5
.gitmodules vendored
View File

@ -193,7 +193,7 @@
url = https://github.com/danlark1/miniselect
[submodule "contrib/rocksdb"]
path = contrib/rocksdb
url = https://github.com/ClickHouse-Extras/rocksdb.git
url = https://github.com/ClickHouse-Extras/rocksdb.git
[submodule "contrib/xz"]
path = contrib/xz
url = https://github.com/xz-mirror/xz
@ -231,3 +231,6 @@
[submodule "contrib/sqlite-amalgamation"]
path = contrib/sqlite-amalgamation
url = https://github.com/azadkuh/sqlite-amalgamation
[submodule "contrib/s2geometry"]
path = contrib/s2geometry
url = https://github.com/google/s2geometry

View File

@ -1,3 +1,4 @@
# Third-party libraries may have substandard code.
# Put all targets defined here and in added subfolders under "contrib/" folder in GUI-based IDEs by default.
# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they will
@ -10,10 +11,8 @@ else ()
endif ()
unset (_current_dir_name)
# Third-party libraries may have substandard code.
# Also remove a possible source of nondeterminism.
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w -D__DATE__= -D__TIME__= -D__TIMESTAMP__=")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w -D__DATE__= -D__TIME__= -D__TIMESTAMP__=")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w")
if (WITH_COVERAGE)
set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE})
@ -333,3 +332,4 @@ if (USE_SQLITE)
add_subdirectory(sqlite-cmake)
endif()
add_subdirectory(s2geometry-cmake)

View File

@ -0,0 +1,116 @@
set(S2_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/s2geometry/src/s2")
set(S2_SRCS
"${S2_LIBRARY_DIR}/base/stringprintf.cc"
"${S2_LIBRARY_DIR}/base/strtoint.cc"
"${S2_LIBRARY_DIR}/encoded_s2cell_id_vector.cc"
"${S2_LIBRARY_DIR}/encoded_s2point_vector.cc"
"${S2_LIBRARY_DIR}/encoded_s2shape_index.cc"
"${S2_LIBRARY_DIR}/encoded_string_vector.cc"
"${S2_LIBRARY_DIR}/id_set_lexicon.cc"
"${S2_LIBRARY_DIR}/mutable_s2shape_index.cc"
"${S2_LIBRARY_DIR}/r2rect.cc"
"${S2_LIBRARY_DIR}/s1angle.cc"
"${S2_LIBRARY_DIR}/s1chord_angle.cc"
"${S2_LIBRARY_DIR}/s1interval.cc"
"${S2_LIBRARY_DIR}/s2boolean_operation.cc"
"${S2_LIBRARY_DIR}/s2builder.cc"
"${S2_LIBRARY_DIR}/s2builder_graph.cc"
"${S2_LIBRARY_DIR}/s2builderutil_closed_set_normalizer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_find_polygon_degeneracies.cc"
"${S2_LIBRARY_DIR}/s2builderutil_lax_polygon_layer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_s2point_vector_layer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_s2polygon_layer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_s2polyline_layer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_s2polyline_vector_layer.cc"
"${S2_LIBRARY_DIR}/s2builderutil_snap_functions.cc"
"${S2_LIBRARY_DIR}/s2cap.cc"
"${S2_LIBRARY_DIR}/s2cell.cc"
"${S2_LIBRARY_DIR}/s2cell_id.cc"
"${S2_LIBRARY_DIR}/s2cell_index.cc"
"${S2_LIBRARY_DIR}/s2cell_union.cc"
"${S2_LIBRARY_DIR}/s2centroids.cc"
"${S2_LIBRARY_DIR}/s2closest_cell_query.cc"
"${S2_LIBRARY_DIR}/s2closest_edge_query.cc"
"${S2_LIBRARY_DIR}/s2closest_point_query.cc"
"${S2_LIBRARY_DIR}/s2contains_vertex_query.cc"
"${S2_LIBRARY_DIR}/s2convex_hull_query.cc"
"${S2_LIBRARY_DIR}/s2coords.cc"
"${S2_LIBRARY_DIR}/s2crossing_edge_query.cc"
"${S2_LIBRARY_DIR}/s2debug.cc"
"${S2_LIBRARY_DIR}/s2earth.cc"
"${S2_LIBRARY_DIR}/s2edge_clipping.cc"
"${S2_LIBRARY_DIR}/s2edge_crosser.cc"
"${S2_LIBRARY_DIR}/s2edge_crossings.cc"
"${S2_LIBRARY_DIR}/s2edge_distances.cc"
"${S2_LIBRARY_DIR}/s2edge_tessellator.cc"
"${S2_LIBRARY_DIR}/s2error.cc"
"${S2_LIBRARY_DIR}/s2furthest_edge_query.cc"
"${S2_LIBRARY_DIR}/s2latlng.cc"
"${S2_LIBRARY_DIR}/s2latlng_rect.cc"
"${S2_LIBRARY_DIR}/s2latlng_rect_bounder.cc"
"${S2_LIBRARY_DIR}/s2lax_loop_shape.cc"
"${S2_LIBRARY_DIR}/s2lax_polygon_shape.cc"
"${S2_LIBRARY_DIR}/s2lax_polyline_shape.cc"
"${S2_LIBRARY_DIR}/s2loop.cc"
"${S2_LIBRARY_DIR}/s2loop_measures.cc"
"${S2_LIBRARY_DIR}/s2measures.cc"
"${S2_LIBRARY_DIR}/s2metrics.cc"
"${S2_LIBRARY_DIR}/s2max_distance_targets.cc"
"${S2_LIBRARY_DIR}/s2min_distance_targets.cc"
"${S2_LIBRARY_DIR}/s2padded_cell.cc"
"${S2_LIBRARY_DIR}/s2point_compression.cc"
"${S2_LIBRARY_DIR}/s2point_region.cc"
"${S2_LIBRARY_DIR}/s2pointutil.cc"
"${S2_LIBRARY_DIR}/s2polygon.cc"
"${S2_LIBRARY_DIR}/s2polyline.cc"
"${S2_LIBRARY_DIR}/s2polyline_alignment.cc"
"${S2_LIBRARY_DIR}/s2polyline_measures.cc"
"${S2_LIBRARY_DIR}/s2polyline_simplifier.cc"
"${S2_LIBRARY_DIR}/s2predicates.cc"
"${S2_LIBRARY_DIR}/s2projections.cc"
"${S2_LIBRARY_DIR}/s2r2rect.cc"
"${S2_LIBRARY_DIR}/s2region.cc"
"${S2_LIBRARY_DIR}/s2region_term_indexer.cc"
"${S2_LIBRARY_DIR}/s2region_coverer.cc"
"${S2_LIBRARY_DIR}/s2region_intersection.cc"
"${S2_LIBRARY_DIR}/s2region_union.cc"
"${S2_LIBRARY_DIR}/s2shape_index.cc"
"${S2_LIBRARY_DIR}/s2shape_index_buffered_region.cc"
"${S2_LIBRARY_DIR}/s2shape_index_measures.cc"
"${S2_LIBRARY_DIR}/s2shape_measures.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_build_polygon_boundaries.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_coding.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_contains_brute_force.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_edge_iterator.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_get_reference_point.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_range_iterator.cc"
"${S2_LIBRARY_DIR}/s2shapeutil_visit_crossing_edge_pairs.cc"
"${S2_LIBRARY_DIR}/s2text_format.cc"
"${S2_LIBRARY_DIR}/s2wedge_relations.cc"
"${S2_LIBRARY_DIR}/strings/ostringstream.cc"
"${S2_LIBRARY_DIR}/strings/serialize.cc"
"${S2_LIBRARY_DIR}/third_party/absl/base/dynamic_annotations.cc"
"${S2_LIBRARY_DIR}/third_party/absl/base/internal/raw_logging.cc"
"${S2_LIBRARY_DIR}/third_party/absl/base/internal/throw_delegate.cc"
"${S2_LIBRARY_DIR}/third_party/absl/numeric/int128.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/ascii.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/match.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/numbers.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/str_cat.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/str_split.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/string_view.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/strip.cc"
"${S2_LIBRARY_DIR}/third_party/absl/strings/internal/memutil.cc"
"${S2_LIBRARY_DIR}/util/bits/bit-interleave.cc"
"${S2_LIBRARY_DIR}/util/bits/bits.cc"
"${S2_LIBRARY_DIR}/util/coding/coder.cc"
"${S2_LIBRARY_DIR}/util/coding/varint.cc"
"${S2_LIBRARY_DIR}/util/math/exactfloat/exactfloat.cc"
"${S2_LIBRARY_DIR}/util/math/mathutil.cc"
"${S2_LIBRARY_DIR}/util/units/length-units.cc"
)
add_library(s2 ${S2_SRCS})
target_include_directories(s2 SYSTEM PUBLIC "${S2_SOURCE_DIR}/")

View File

@ -124,3 +124,5 @@ endif()
# Signed integer overflow on user-provided data inside boost::geometry - ignore.
set_source_files_properties("pointInPolygon.cpp" PROPERTIES COMPILE_FLAGS -fno-sanitize=signed-integer-overflow)
target_link_libraries(clickhouse_functions PRIVATE ${S2_SOURCE_DIR})

View File

@ -0,0 +1,108 @@
#include "config_functions.h"
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include "../contrib/s2geometry/src/s2/s2latlng.h"
#include "../contrib/s2geometry/src/s2/s2cell_id.h"
class S2CellId;
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// TODO: Comment this
class FunctionS2CellsIntersect : public IFunction
{
public:
static constexpr auto name = "S2CellsIntersect";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionS2CellsIntersect>();
}
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 2) {
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
arg = arguments[1].get();
if (!WhichDataType(arg).isUInt64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_id_first = arguments[0].column.get();
const auto * col_id_second = arguments[1].column.get();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const UInt64 id_first = col_id_first->getInt(row);
const UInt64 id_second = col_id_second->getInt(row);
dst_data[row] = S2CellId(id_first).intersects(S2CellId(id_second));
}
return dst;
}
};
}
void registerFunctionS2CellsIntersect(FunctionFactory & factory)
{
factory.registerFunction<FunctionS2CellsIntersect>();
}
}

View File

@ -0,0 +1,109 @@
#include "config_functions.h"
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include "../contrib/s2geometry/src/s2/s2latlng.h"
#include "../contrib/s2geometry/src/s2/s2cell_id.h"
#include "../contrib/s2geometry/src/s2/s2point.h"
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// TODO: Comment this
class FunctionS2GetNeighbors : public IFunction
{
public:
static constexpr auto name = "S2GetNeighbors";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionS2GetNeighbors>();
}
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 1) {
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_id = arguments[0].column.get();
auto dst = ColumnArray::create(ColumnUInt64::create());
auto & dst_data = dst->getData();
auto & dst_offsets = dst->getOffsets();
dst_offsets.resize(input_rows_count);
auto current_offset = 0;
for (const auto row : ext::range(0, input_rows_count))
{
const UInt64 id = col_id->getUInt(row);
S2CellId cell_id(id);
S2CellId neighbors[4];
cell_id.GetEdgeNeighbors(neighbors);
dst_data.reserve(dst_data.size() + 4);
for (int i = 0; i < 4; ++i)
{
++current_offset;
dst_data.insert(neighbors[i].id());
}
dst_offsets[row] = current_offset;
}
return dst;
}
};
}
void registerFunctionS2GetNeighbors(FunctionFactory & factory)
{
factory.registerFunction<FunctionS2GetNeighbors>();
}
}

110
src/Functions/S2ToGeo.cpp Normal file
View File

@ -0,0 +1,110 @@
#include "config_functions.h"
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include "../contrib/s2geometry/src/s2/s2latlng.h"
#include "../contrib/s2geometry/src/s2/s2cell_id.h"
class S2CellId;
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// TODO: Comment this
class FunctionS2ToGeo : public IFunction
{
public:
static constexpr auto name = "S2ToGeo";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionS2ToGeo>();
}
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 1) {
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
DataTypePtr element = std::make_shared<DataTypeUInt64>();
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_id = arguments[0].column.get();
auto col_res_first = ColumnUInt32::create();
auto col_res_second = ColumnUInt32::create();
auto & vec_res_first = col_res_first->getData();
vec_res_first.resize(input_rows_count);
auto & vec_res_second = col_res_second->getData();
vec_res_second.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const UInt64 id = col_id->getUInt(row);
S2CellId cell_id(id);
S2Point point = cell_id.ToPoint();
S2LatLng ll(point);
vec_res_first[row] = ll.lat().degrees();
vec_res_second[row] = ll.lat().degrees();
}
return ColumnTuple::create(Columns{std::move(col_res_first), std::move(col_res_second)});
}
};
}
void registerFunctionGeoToS2(FunctionFactory & factory)
{
factory.registerFunction<FunctionS2ToGeo>();
}
}

View File

@ -0,0 +1,108 @@
#include "config_functions.h"
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include "../contrib/s2geometry/src/s2/s2latlng.h"
#include "../contrib/s2geometry/src/s2/s2cell_id.h"
class S2CellId;
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// TODO: Comment this
class FunctionDegreesToS2 : public IFunction
{
public:
static constexpr auto name = "degreesToS2";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionDegreesToS2>();
}
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 2) {
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isFloat64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
arg = arguments[1].get();
if (!WhichDataType(arg).isFloat64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_lon = arguments[0].column.get();
const auto * col_lat = arguments[1].column.get();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const Float64 lon = col_lon->getFloat64(row);
const Float64 lat = col_lat->getFloat64(row);
S2LatLng lat_lng = S2LatLng::FromDegrees(lat, lon);
S2CellId id(lat_lng);
dst_data[row] = id.id();
}
return dst;
}
};
}
void registerFunctionDegreesToS2(FunctionFactory & factory)
{
factory.registerFunction<FunctionDegreesToS2>();
}
}

View File

@ -0,0 +1,110 @@
#include "config_functions.h"
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include "../contrib/s2geometry/src/s2/s2latlng.h"
#include "../contrib/s2geometry/src/s2/s2cell_id.h"
class S2CellId;
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// TODO: Comment this
class FunctionRadiansToS2 : public IFunction
{
public:
static constexpr auto name = "radiansToS2";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionRadiansToS2>();
}
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments != 2) {
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isFloat64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
arg = arguments[1].get();
if (!WhichDataType(arg).isFloat64()) {
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_lon = arguments[0].column.get();
const auto * col_lat = arguments[1].column.get();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const double lon = col_lon->getFloat64(row);
double lat = col_lat->getFloat64(row);
lat = lon;
S2LatLng lat_lng = S2LatLng::FromRadians(lat, lon);
S2CellId id(lat_lng);
dst_data[row] = id.id();
}
return dst;
}
};
}
void registerRadiansDegreesToS2(FunctionFactory & factory)
{
factory.registerFunction<FunctionRadiansToS2>();
}
}

View File

@ -42,6 +42,12 @@ void registerFunctionH3ToString(FunctionFactory &);
void registerFunctionH3HexAreaM2(FunctionFactory &);
#endif
void registerFunctionDegreesToS2(FunctionFactory &);
void registerFunctionRadiansToS2(FunctionFactory &);
void registerFunctionS2GetNeighbors(FunctionFactory &);
void registerFunctionS2ToGeo(FunctionFactory &);
void registerFunctionS2CellsIntersect(FunctionFactory &);
void registerFunctionsGeo(FunctionFactory & factory)
{
@ -79,6 +85,12 @@ void registerFunctionsGeo(FunctionFactory & factory)
registerFunctionH3ToString(factory);
registerFunctionH3HexAreaM2(factory);
#endif
registerFunctionDegreesToS2(factory);
registerFunctionRadiansToS2(factory);
registerFunctionS2GetNeighbors(factory);
registerFunctionS2ToGeo(factory);
registerFunctionS2CellsIntersect(factory);
}
}