Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2019-07-01 02:54:07 +03:00
commit f6c2815134
22 changed files with 699 additions and 334 deletions

3
.gitmodules vendored
View File

@ -76,6 +76,9 @@
[submodule "contrib/brotli"]
path = contrib/brotli
url = https://github.com/google/brotli.git
[submodule "contrib/h3"]
path = contrib/h3
url = https://github.com/uber/h3
[submodule "contrib/hyperscan"]
path = contrib/hyperscan
url = https://github.com/ClickHouse-Extras/hyperscan.git

View File

@ -341,6 +341,7 @@ include (cmake/find_libgsasl.cmake)
include (cmake/find_rdkafka.cmake)
include (cmake/find_capnp.cmake)
include (cmake/find_llvm.cmake)
include (cmake/find_h3.cmake)
include (cmake/find_cpuid.cmake) # Freebsd, bundled
if (NOT USE_CPUID)
include (cmake/find_cpuinfo.cmake) # Debian
@ -418,6 +419,7 @@ if (GLIBC_COMPATIBILITY)
add_glibc_compat(kj)
add_glibc_compat(simdjson)
add_glibc_compat(apple_rt)
add_glibc_compat(h3)
add_glibc_compat(re2)
add_glibc_compat(re2_st)
add_glibc_compat(hs_compile_shared)

19
cmake/find_h3.cmake Normal file
View File

@ -0,0 +1,19 @@
option (USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED})
set (H3_INCLUDE_PATHS /usr/local/include/h3)
if (USE_INTERNAL_H3_LIBRARY)
set (H3_LIBRARY h3)
set (H3_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include)
else ()
find_library (H3_LIBRARY h3)
find_path (H3_INCLUDE_DIR NAMES h3api.h PATHS ${H3_INCLUDE_PATHS})
endif ()
if (H3_LIBRARY AND H3_INCLUDE_DIR)
set (USE_H3 1)
else ()
set (USE_H3 0)
endif ()
message (STATUS "Using h3=${USE_H3}: ${H3_INCLUDE_DIR} : ${H3_LIBRARY}")

View File

@ -106,6 +106,10 @@ if (USE_INTERNAL_CPUID_LIBRARY)
add_subdirectory (libcpuid)
endif ()
if (USE_INTERNAL_H3_LIBRARY)
add_subdirectory(h3-cmake)
endif ()
if (USE_INTERNAL_SSL_LIBRARY)
if (NOT MAKE_STATIC_LIBRARIES)
set (BUILD_SHARED 1)

1
contrib/h3 vendored Submodule

@ -0,0 +1 @@
Subproject commit 6cfd649e8c0d3ed913e8aae928a669fc3b8a2365

View File

@ -0,0 +1,27 @@
set(H3_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib)
set(H3_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/h3/src/h3lib)
set(SRCS
${H3_SOURCE_DIR}/lib/algos.c
${H3_SOURCE_DIR}/lib/baseCells.c
${H3_SOURCE_DIR}/lib/bbox.c
${H3_SOURCE_DIR}/lib/coordijk.c
${H3_SOURCE_DIR}/lib/faceijk.c
${H3_SOURCE_DIR}/lib/geoCoord.c
${H3_SOURCE_DIR}/lib/h3Index.c
${H3_SOURCE_DIR}/lib/h3UniEdge.c
${H3_SOURCE_DIR}/lib/linkedGeo.c
${H3_SOURCE_DIR}/lib/localij.c
${H3_SOURCE_DIR}/lib/mathExtensions.c
${H3_SOURCE_DIR}/lib/polygon.c
${H3_SOURCE_DIR}/lib/vec2d.c
${H3_SOURCE_DIR}/lib/vec3d.c
${H3_SOURCE_DIR}/lib/vertexGraph.c
)
configure_file(${H3_SOURCE_DIR}/include/h3api.h.in ${H3_BINARY_DIR}/include/h3api.h)
add_library(h3 ${SRCS})
target_include_directories(h3 SYSTEM PUBLIC ${H3_SOURCE_DIR}/include)
target_include_directories(h3 SYSTEM PUBLIC ${H3_BINARY_DIR}/include)
target_compile_definitions(h3 PRIVATE H3_HAVE_VLA)

View File

@ -65,6 +65,11 @@ if(USE_XXHASH)
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${XXHASH_INCLUDE_DIR})
endif()
if (USE_H3)
target_link_libraries(clickhouse_functions PRIVATE ${H3_LIBRARY})
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${H3_INCLUDE_DIR})
endif()
if(USE_HYPERSCAN)
target_link_libraries(clickhouse_functions PRIVATE ${HYPERSCAN_LIBRARY})
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${HYPERSCAN_INCLUDE_DIR})

View File

@ -8,3 +8,4 @@
#cmakedefine01 USE_HYPERSCAN
#cmakedefine01 USE_SIMDJSON
#cmakedefine01 USE_RAPIDJSON
#cmakedefine01 USE_H3

View File

@ -0,0 +1,108 @@
#include "config_functions.h"
#if USE_H3
#include <array>
#include <math.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
extern "C"
{
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wdocumentation"
#endif
#include <h3api.h>
#ifdef __clang__
#pragma clang diagnostic pop
#endif
}
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/// Implements the function geoToH3 which takes 3 arguments (latitude, longitude and h3 resolution)
/// and returns h3 index of this point
class FunctionGeoToH3 : public IFunction
{
public:
static constexpr auto name = "geoToH3";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeoToH3>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
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);
arg = arguments[2].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(3) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_lon = block.getByPosition(arguments[0]).column.get();
const auto col_lat = block.getByPosition(arguments[1]).column.get();
const auto col_res = block.getByPosition(arguments[2]).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);
const double lat = col_lat->getFloat64(row);
const UInt8 res = col_res->getUInt(row);
GeoCoord coord;
coord.lon = H3_EXPORT(degsToRads)(lon);
coord.lat = H3_EXPORT(degsToRads)(lat);
H3Index hindex = H3_EXPORT(geoToH3)(&coord, res);
dst_data[row] = hindex;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionGeoToH3(FunctionFactory & factory)
{
factory.registerFunction<FunctionGeoToH3>();
}
}
#endif

View File

@ -0,0 +1,99 @@
#include <Functions/FunctionFactory.h>
#include <Functions/GeoUtils.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <string>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
// geohashDecode(string) => (lon float64, lat float64)
class FunctionGeohashDecode : public IFunction
{
public:
static constexpr auto name = "geohashDecode";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashDecode>(); }
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
{
validateArgumentType(*this, arguments, 0, isStringOrFixedString, "string or fixed string");
return std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeFloat64>(), std::make_shared<DataTypeFloat64>()},
Strings{"longitude", "latitude"});
}
template <typename ColumnTypeEncoded>
bool tryExecute(const IColumn * encoded_column, ColumnPtr & result_column)
{
const auto * encoded = checkAndGetColumn<ColumnTypeEncoded>(encoded_column);
if (!encoded)
return false;
const size_t count = encoded->size();
auto latitude = ColumnFloat64::create(count);
auto longitude = ColumnFloat64::create(count);
ColumnFloat64::Container & lon_data = longitude->getData();
ColumnFloat64::Container & lat_data = latitude->getData();
for (size_t i = 0; i < count; ++i)
{
StringRef encoded_string = encoded->getDataAt(i);
GeoUtils::geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]);
}
MutableColumns result;
result.emplace_back(std::move(longitude));
result.emplace_back(std::move(latitude));
result_column = ColumnTuple::create(std::move(result));
return true;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * encoded = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
if (tryExecute<ColumnString>(encoded, res_column) ||
tryExecute<ColumnFixedString>(encoded, res_column))
return;
throw Exception("Unsupported argument type:" + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
void registerFunctionGeohashDecode(FunctionFactory & factory)
{
factory.registerFunction<FunctionGeohashDecode>();
}
}

View File

@ -0,0 +1,136 @@
#include <Functions/FunctionFactory.h>
#include <Functions/GeoUtils.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <string>
#define GEOHASH_MAX_TEXT_LENGTH 16
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
}
// geohashEncode(lon float32/64, lat float32/64, length UInt8) => string
class FunctionGeohashEncode : public IFunction
{
public:
static constexpr auto name = "geohashEncode";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashEncode>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
validateArgumentType(*this, arguments, 0, isFloat, "float");
validateArgumentType(*this, arguments, 1, isFloat, "float");
if (arguments.size() == 3)
{
validateArgumentType(*this, arguments, 2, isInteger, "integer");
}
if (arguments.size() > 3)
{
throw Exception("Too many arguments for function " + getName() +
" expected at most 3",
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION);
}
return std::make_shared<DataTypeString>();
}
template <typename LonType, typename LatType>
bool tryExecute(const IColumn * lon_column, const IColumn * lat_column, UInt64 precision_value, ColumnPtr & result)
{
const ColumnVector<LonType> * longitude = checkAndGetColumn<ColumnVector<LonType>>(lon_column);
const ColumnVector<LatType> * latitude = checkAndGetColumn<ColumnVector<LatType>>(lat_column);
if (!latitude || !longitude)
return false;
auto col_str = ColumnString::create();
ColumnString::Chars & out_vec = col_str->getChars();
ColumnString::Offsets & out_offsets = col_str->getOffsets();
const size_t size = lat_column->size();
out_offsets.resize(size);
out_vec.resize(size * (GEOHASH_MAX_TEXT_LENGTH + 1));
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
for (size_t i = 0; i < size; ++i)
{
const Float64 longitude_value = longitude->getElement(i);
const Float64 latitude_value = latitude->getElement(i);
const size_t encoded_size = GeoUtils::geohashEncode(longitude_value, latitude_value, precision_value, pos);
pos += encoded_size;
*pos = '\0';
out_offsets[i] = ++pos - begin;
}
out_vec.resize(pos - begin);
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
result = std::move(col_str);
return true;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * longitude = block.getByPosition(arguments[0]).column.get();
const IColumn * latitude = block.getByPosition(arguments[1]).column.get();
const UInt64 precision_value = std::min<UInt64>(GEOHASH_MAX_TEXT_LENGTH,
arguments.size() == 3 ? block.getByPosition(arguments[2]).column->get64(0) : GEOHASH_MAX_TEXT_LENGTH);
ColumnPtr & res_column = block.getByPosition(result).column;
if (tryExecute<Float32, Float32>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float64, Float32>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float32, Float64>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float64, Float64>(longitude, latitude, precision_value, res_column))
return;
std::string arguments_description;
for (size_t i = 0; i < arguments.size(); ++i)
{
if (i != 0)
arguments_description += ", ";
arguments_description += block.getByPosition(arguments[i]).column->getName();
}
throw Exception("Unsupported argument types: " + arguments_description +
+ " for function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
void registerFunctionGeohashEncode(FunctionFactory & factory)
{
factory.registerFunction<FunctionGeohashEncode>();
}
}

View File

@ -0,0 +1,166 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Common/typeid_cast.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <ext/range.h>
#include <math.h>
#include <array>
#define DEGREES_IN_RADIANS (M_PI / 180.0)
#define EARTH_RADIUS_IN_METERS 6372797.560856
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
}
static inline Float64 degToRad(Float64 angle) { return angle * DEGREES_IN_RADIANS; }
/**
* The function calculates distance in meters between two points on Earth specified by longitude and latitude in degrees.
* The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance.
* Throws exception when one or several input values are not within reasonable bounds.
* Latitude must be in [-90, 90], longitude must be [-180, 180]
*
*/
class FunctionGreatCircleDistance : public IFunction
{
public:
static constexpr auto name = "greatCircleDistance";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGreatCircleDistance>(); }
private:
enum class instr_type : uint8_t
{
get_float_64,
get_const_float_64
};
using instr_t = std::pair<instr_type, const IColumn *>;
using instrs_t = std::array<instr_t, 4>;
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 4; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
for (const auto arg_idx : ext::range(0, arguments.size()))
{
const auto arg = arguments[arg_idx].get();
if (!WhichDataType(arg).isFloat64())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeFloat64>();
}
instrs_t getInstructions(const Block & block, const ColumnNumbers & arguments, bool & out_const)
{
instrs_t result;
out_const = true;
for (const auto arg_idx : ext::range(0, arguments.size()))
{
const auto column = block.getByPosition(arguments[arg_idx]).column.get();
if (const auto col = checkAndGetColumn<ColumnVector<Float64>>(column))
{
out_const = false;
result[arg_idx] = instr_t{instr_type::get_float_64, col};
}
else if (const auto col_const = checkAndGetColumnConst<ColumnVector<Float64>>(column))
{
result[arg_idx] = instr_t{instr_type::get_const_float_64, col_const};
}
else
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return result;
}
/// https://en.wikipedia.org/wiki/Great-circle_distance
Float64 greatCircleDistance(Float64 lon1Deg, Float64 lat1Deg, Float64 lon2Deg, Float64 lat2Deg)
{
if (lon1Deg < -180 || lon1Deg > 180 ||
lon2Deg < -180 || lon2Deg > 180 ||
lat1Deg < -90 || lat1Deg > 90 ||
lat2Deg < -90 || lat2Deg > 90)
{
throw Exception("Arguments values out of bounds for function " + getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
Float64 lon1Rad = degToRad(lon1Deg);
Float64 lat1Rad = degToRad(lat1Deg);
Float64 lon2Rad = degToRad(lon2Deg);
Float64 lat2Rad = degToRad(lat2Deg);
Float64 u = sin((lat2Rad - lat1Rad) / 2);
Float64 v = sin((lon2Rad - lon1Rad) / 2);
return 2.0 * EARTH_RADIUS_IN_METERS * asin(sqrt(u * u + cos(lat1Rad) * cos(lat2Rad) * v * v));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto size = input_rows_count;
bool result_is_const{};
auto instrs = getInstructions(block, arguments, result_is_const);
if (result_is_const)
{
const auto & colLon1 = static_cast<const ColumnConst *>(block.getByPosition(arguments[0]).column.get())->getValue<Float64>();
const auto & colLat1 = static_cast<const ColumnConst *>(block.getByPosition(arguments[1]).column.get())->getValue<Float64>();
const auto & colLon2 = static_cast<const ColumnConst *>(block.getByPosition(arguments[2]).column.get())->getValue<Float64>();
const auto & colLat2 = static_cast<const ColumnConst *>(block.getByPosition(arguments[3]).column.get())->getValue<Float64>();
Float64 res = greatCircleDistance(colLon1, colLat1, colLon2, colLat2);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, res);
}
else
{
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
Float64 vals[instrs.size()];
for (const auto row : ext::range(0, size))
{
for (const auto idx : ext::range(0, instrs.size()))
{
if (instr_type::get_float_64 == instrs[idx].first)
vals[idx] = static_cast<const ColumnVector<Float64> *>(instrs[idx].second)->getData()[row];
else if (instr_type::get_const_float_64 == instrs[idx].first)
vals[idx] = static_cast<const ColumnConst *>(instrs[idx].second)->getValue<Float64>();
else
throw Exception{"Unknown instruction type in implementation of greatCircleDistance function", ErrorCodes::LOGICAL_ERROR};
}
dst_data[row] = greatCircleDistance(vals[0], vals[1], vals[2], vals[3]);
}
block.getByPosition(result).column = std::move(dst);
}
}
};
void registerFunctionGreatCircleDistance(FunctionFactory & factory)
{
factory.registerFunction<FunctionGreatCircleDistance>();
}
}

View File

@ -1,17 +1,11 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Common/typeid_cast.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <ext/range.h>
#include <math.h>
#include <array>
#define DEGREES_IN_RADIANS (M_PI / 180.0)
#define EARTH_RADIUS_IN_METERS 6372797.560856
namespace DB
@ -19,148 +13,11 @@ namespace DB
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
}
static inline Float64 degToRad(Float64 angle) { return angle * DEGREES_IN_RADIANS; }
static inline Float64 radToDeg(Float64 angle) { return angle / DEGREES_IN_RADIANS; }
/**
* The function calculates distance in meters between two points on Earth specified by longitude and latitude in degrees.
* The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance.
* Throws exception when one or several input values are not within reasonable bounds.
* Latitude must be in [-90, 90], longitude must be [-180, 180]
*
*/
class FunctionGreatCircleDistance : public IFunction
{
public:
static constexpr auto name = "greatCircleDistance";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGreatCircleDistance>(); }
private:
enum class instr_type : uint8_t
{
get_float_64,
get_const_float_64
};
using instr_t = std::pair<instr_type, const IColumn *>;
using instrs_t = std::array<instr_t, 4>;
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 4; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
for (const auto arg_idx : ext::range(0, arguments.size()))
{
const auto arg = arguments[arg_idx].get();
if (!WhichDataType(arg).isFloat64())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeFloat64>();
}
instrs_t getInstructions(const Block & block, const ColumnNumbers & arguments, bool & out_const)
{
instrs_t result;
out_const = true;
for (const auto arg_idx : ext::range(0, arguments.size()))
{
const auto column = block.getByPosition(arguments[arg_idx]).column.get();
if (const auto col = checkAndGetColumn<ColumnVector<Float64>>(column))
{
out_const = false;
result[arg_idx] = instr_t{instr_type::get_float_64, col};
}
else if (const auto col_const = checkAndGetColumnConst<ColumnVector<Float64>>(column))
{
result[arg_idx] = instr_t{instr_type::get_const_float_64, col_const};
}
else
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return result;
}
/// https://en.wikipedia.org/wiki/Great-circle_distance
Float64 greatCircleDistance(Float64 lon1Deg, Float64 lat1Deg, Float64 lon2Deg, Float64 lat2Deg)
{
if (lon1Deg < -180 || lon1Deg > 180 ||
lon2Deg < -180 || lon2Deg > 180 ||
lat1Deg < -90 || lat1Deg > 90 ||
lat2Deg < -90 || lat2Deg > 90)
{
throw Exception("Arguments values out of bounds for function " + getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
Float64 lon1Rad = degToRad(lon1Deg);
Float64 lat1Rad = degToRad(lat1Deg);
Float64 lon2Rad = degToRad(lon2Deg);
Float64 lat2Rad = degToRad(lat2Deg);
Float64 u = sin((lat2Rad - lat1Rad) / 2);
Float64 v = sin((lon2Rad - lon1Rad) / 2);
return 2.0 * EARTH_RADIUS_IN_METERS * asin(sqrt(u * u + cos(lat1Rad) * cos(lat2Rad) * v * v));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto size = input_rows_count;
bool result_is_const{};
auto instrs = getInstructions(block, arguments, result_is_const);
if (result_is_const)
{
const auto & colLon1 = static_cast<const ColumnConst *>(block.getByPosition(arguments[0]).column.get())->getValue<Float64>();
const auto & colLat1 = static_cast<const ColumnConst *>(block.getByPosition(arguments[1]).column.get())->getValue<Float64>();
const auto & colLon2 = static_cast<const ColumnConst *>(block.getByPosition(arguments[2]).column.get())->getValue<Float64>();
const auto & colLat2 = static_cast<const ColumnConst *>(block.getByPosition(arguments[3]).column.get())->getValue<Float64>();
Float64 res = greatCircleDistance(colLon1, colLat1, colLon2, colLat2);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, res);
}
else
{
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
Float64 vals[instrs.size()];
for (const auto row : ext::range(0, size))
{
for (const auto idx : ext::range(0, instrs.size()))
{
if (instr_type::get_float_64 == instrs[idx].first)
vals[idx] = static_cast<const ColumnVector<Float64> *>(instrs[idx].second)->getData()[row];
else if (instr_type::get_const_float_64 == instrs[idx].first)
vals[idx] = static_cast<const ColumnConst *>(instrs[idx].second)->getValue<Float64>();
else
throw Exception{"Unknown instruction type in implementation of greatCircleDistance function", ErrorCodes::LOGICAL_ERROR};
}
dst_data[row] = greatCircleDistance(vals[0], vals[1], vals[2], vals[3]);
}
block.getByPosition(result).column = std::move(dst);
}
}
};
/**
* The function checks if a point is in one of ellipses in set.
* The number of arguments must be 2 + 4*N where N is the number of ellipses.
@ -177,7 +34,6 @@ private:
class FunctionPointInEllipses : public IFunction
{
public:
static constexpr auto name = "pointInEllipses";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionPointInEllipses>(); }
@ -330,6 +186,10 @@ private:
}
};
void registerFunctionPointInEllipses(FunctionFactory & factory)
{
factory.registerFunction<FunctionPointInEllipses>();
}
#undef DEGREES_IN_RADIANS
}

View File

@ -1,5 +1,4 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsGeo.h>
#include <Functions/GeoUtils.h>
#include <Functions/FunctionHelpers.h>
@ -16,6 +15,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/ExpressionActions.h>
@ -37,6 +37,7 @@ namespace ErrorCodes
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace FunctionPointInPolygonDetail
@ -251,185 +252,6 @@ private:
};
const size_t GEOHASH_MAX_TEXT_LENGTH = 16;
// geohashEncode(lon float32/64, lat float32/64, length UInt8) => string
class FunctionGeohashEncode : public IFunction
{
public:
static constexpr auto name = "geohashEncode";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashEncode>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
validateArgumentType(*this, arguments, 0, isFloat, "float");
validateArgumentType(*this, arguments, 1, isFloat, "float");
if (arguments.size() == 3)
{
validateArgumentType(*this, arguments, 2, isInteger, "integer");
}
if (arguments.size() > 3)
{
throw Exception("Too many arguments for function " + getName() +
" expected at most 3",
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION);
}
return std::make_shared<DataTypeString>();
}
template <typename LonType, typename LatType>
bool tryExecute(const IColumn * lon_column, const IColumn * lat_column, UInt64 precision_value, ColumnPtr & result)
{
const ColumnVector<LonType> * longitude = checkAndGetColumn<ColumnVector<LonType>>(lon_column);
const ColumnVector<LatType> * latitude = checkAndGetColumn<ColumnVector<LatType>>(lat_column);
if (!latitude || !longitude)
return false;
auto col_str = ColumnString::create();
ColumnString::Chars & out_vec = col_str->getChars();
ColumnString::Offsets & out_offsets = col_str->getOffsets();
const size_t size = lat_column->size();
out_offsets.resize(size);
out_vec.resize(size * (GEOHASH_MAX_TEXT_LENGTH + 1));
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
for (size_t i = 0; i < size; ++i)
{
const Float64 longitude_value = longitude->getElement(i);
const Float64 latitude_value = latitude->getElement(i);
const size_t encoded_size = GeoUtils::geohashEncode(longitude_value, latitude_value, precision_value, pos);
pos += encoded_size;
*pos = '\0';
out_offsets[i] = ++pos - begin;
}
out_vec.resize(pos - begin);
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
result = std::move(col_str);
return true;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * longitude = block.getByPosition(arguments[0]).column.get();
const IColumn * latitude = block.getByPosition(arguments[1]).column.get();
const UInt64 precision_value = std::min<UInt64>(GEOHASH_MAX_TEXT_LENGTH,
arguments.size() == 3 ? block.getByPosition(arguments[2]).column->get64(0) : GEOHASH_MAX_TEXT_LENGTH);
ColumnPtr & res_column = block.getByPosition(result).column;
if (tryExecute<Float32, Float32>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float64, Float32>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float32, Float64>(longitude, latitude, precision_value, res_column) ||
tryExecute<Float64, Float64>(longitude, latitude, precision_value, res_column))
return;
const char sep[] = ", ";
std::string arguments_description = "";
for (size_t i = 0; i < arguments.size(); ++i)
{
arguments_description += block.getByPosition(arguments[i]).column->getName() + sep;
}
if (arguments_description.size() > sizeof(sep))
{
arguments_description.erase(arguments_description.size() - sizeof(sep) - 1);
}
throw Exception("Unsupported argument types: " + arguments_description +
+ " for function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
// geohashDecode(string) => (lon float64, lat float64)
class FunctionGeohashDecode : public IFunction
{
public:
static constexpr auto name = "geohashDecode";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashDecode>(); }
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
{
validateArgumentType(*this, arguments, 0, isStringOrFixedString, "string or fixed string");
return std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeFloat64>(), std::make_shared<DataTypeFloat64>()},
Strings{"longitude", "latitude"});
}
template <typename ColumnTypeEncoded>
bool tryExecute(const IColumn * encoded_column, ColumnPtr & result_column)
{
const auto * encoded = checkAndGetColumn<ColumnTypeEncoded>(encoded_column);
if (!encoded)
return false;
const size_t count = encoded->size();
auto latitude = ColumnFloat64::create(count);
auto longitude = ColumnFloat64::create(count);
ColumnFloat64::Container & lon_data = longitude->getData();
ColumnFloat64::Container & lat_data = latitude->getData();
for (size_t i = 0; i < count; ++i)
{
StringRef encoded_string = encoded->getDataAt(i);
GeoUtils::geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]);
}
MutableColumns result;
result.emplace_back(std::move(longitude));
result.emplace_back(std::move(latitude));
result_column = ColumnTuple::create(std::move(result));
return true;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * encoded = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
if (tryExecute<ColumnString>(encoded, res_column) ||
tryExecute<ColumnFixedString>(encoded, res_column))
return;
throw Exception("Unsupported argument type:" + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
template <typename Type>
using Point = boost::geometry::model::d2::point_xy<Type>;
@ -440,13 +262,9 @@ using PointInPolygonWithGrid = GeoUtils::PointInPolygonWithGrid<Type>;
template <>
const char * FunctionPointInPolygon<PointInPolygonWithGrid, true>::name = "pointInPolygon";
void registerFunctionsGeo(FunctionFactory & factory)
void registerFunctionPointInPolygon(FunctionFactory & factory)
{
factory.registerFunction<FunctionGreatCircleDistance>();
factory.registerFunction<FunctionPointInEllipses>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid, true>>();
factory.registerFunction<FunctionGeohashEncode>();
factory.registerFunction<FunctionGeohashDecode>();
}
}

View File

@ -0,0 +1,32 @@
#include "config_functions.h"
namespace DB
{
class FunctionFactory;
void registerFunctionGreatCircleDistance(FunctionFactory & factory);
void registerFunctionPointInEllipses(FunctionFactory & factory);
void registerFunctionPointInPolygon(FunctionFactory & factory);
void registerFunctionGeohashEncode(FunctionFactory & factory);
void registerFunctionGeohashDecode(FunctionFactory & factory);
#if USE_H3
void registerFunctionGeoToH3(FunctionFactory &);
#endif
void registerFunctionsGeo(FunctionFactory & factory)
{
registerFunctionGreatCircleDistance(factory);
registerFunctionPointInEllipses(factory);
registerFunctionPointInPolygon(factory);
registerFunctionGeohashEncode(factory);
registerFunctionGeohashDecode(factory);
#if USE_H3
registerFunctionGeoToH3(factory);
#endif
}
}

View File

@ -40,6 +40,7 @@ const char * auto_config_build[]
"USE_MIMALLOC", "@USE_MIMALLOC@",
"USE_UNWIND", "@USE_UNWIND@",
"USE_ICU", "@USE_ICU@",
"USE_H3", "@USE_H3@",
"USE_MYSQL", "@USE_MYSQL@",
"USE_RE2_ST", "@USE_RE2_ST@",
"USE_VECTORCLASS", "@USE_VECTORCLASS@",

View File

@ -0,0 +1,14 @@
<test>
<type>once</type>
<stop_conditions>
<any_of>
<!-- This is only for infinite running query. -->
<average_speed_not_changing_for_ms>2000</average_speed_not_changing_for_ms>
<total_time_ms>10000</total_time_ms>
</any_of>
</stop_conditions>
<!-- Moscow coordinates, maximum precision. NOTE: H3 library is extraordinary slow. -->
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geoToH3(37.62 + rand(1) / 0x100000000, 55.75 + rand(2) / 0x100000000, 15))</query>
</test>

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,20 @@
644325529094369568
639821928864584823
644325528491955313
644325528491955313
644325528627451570
644325529094369568
644325528491955313
644325528491955313
644325528491955313
644325528627451570
644325529094369568
55.720762 37.598135 644325528491955313
55.720762 37.598135 644325528491955313
55.72076201 37.598135 644325528491955313
55.763241 37.660183 644325528627451570
55.77922738 37.63098076 644325529094369568
639821928864584823 1
644325528491955313 2
644325528627451570 1
644325529094369568 1

View File

@ -0,0 +1,19 @@
USE test;
DROP TABLE IF EXISTS table1;
CREATE TABLE table1 (lat Float64, lon Float64, resolution UInt8) ENGINE = Memory;
INSERT INTO table1 VALUES(55.77922738, 37.63098076, 15);
INSERT INTO table1 VALUES(55.76324100, 37.66018300, 15);
INSERT INTO table1 VALUES(55.72076200, 37.59813500, 15);
INSERT INTO table1 VALUES(55.72076201, 37.59813500, 15);
INSERT INTO table1 VALUES(55.72076200, 37.59813500, 14);
select geoToH3(37.63098076, 55.77922738, 15);
select geoToH3(lon, lat, resolution) from table1 order by lat, lon, resolution;
select geoToH3(lon, lat, 15) AS k from table1 order by lat, lon, k;
select lat, lon, geoToH3(lon, lat, 15) AS k from table1 order by lat, lon, k;
select geoToH3(lon, lat, resolution) AS k, count(*) from table1 group by k order by k;
DROP TABLE table1

View File

@ -70,6 +70,5 @@ RUN apt-get --allow-unauthenticated update -y \
gperf \
alien
COPY build.sh /
CMD ["/bin/bash", "/build.sh"]

View File

@ -152,4 +152,35 @@ SELECT geohashDecode('ezs42') AS res
└─────────────────────────────────┘
```
## geoToH3
Получает H3 индекс точки (lon, lat) с заданным разрешением
```
geoToH3(lon, lat, resolution)
```
**Входные значения**
- `lon` - географическая долгота. Тип данных — [Float64](../../data_types/float.md).
- `lat` - географическая широта. Тип данных — [Float64](../../data_types/float.md).
- `resolution` - требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значение — `[0, 15]`.
**Возвращаемые значения**
Возвращает значение с типом [UInt64] (../../data_types/int_uint.md).
`0` в случае ошибки.
Иначе возвращается индексный номер шестиугольника.
**Пример**
``` sql
SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index
```
```
┌────────────h3Index─┐
│ 644325524701193974 │
└────────────────────┘
```
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/geo/) <!--hide-->