From 2fb5addc4095a5093846732076ae61a5b3947fbd Mon Sep 17 00:00:00 2001 From: Ivan Remen Date: Mon, 25 Mar 2019 17:34:52 +0300 Subject: [PATCH] H3 library integration --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find_h3.cmake | 17 ++ contrib/CMakeLists.txt | 5 + dbms/src/Functions/CMakeLists.txt | 6 + dbms/src/Functions/geoToH3.cpp | 171 ++++++++++++++++++ dbms/src/Functions/registerFunctions.cpp | 2 + .../queries/0_stateless/00746_sql_fuzzy.pl | 2 +- .../0_stateless/00926_geo_to_h3.reference | 20 ++ .../queries/0_stateless/00926_geo_to_h3.sql | 19 ++ docs/ru/query_language/functions/geo.md | 33 ++++ 11 files changed, 278 insertions(+), 1 deletion(-) create mode 100644 cmake/find_h3.cmake create mode 100644 dbms/src/Functions/geoToH3.cpp create mode 100644 dbms/tests/queries/0_stateless/00926_geo_to_h3.reference create mode 100644 dbms/tests/queries/0_stateless/00926_geo_to_h3.sql diff --git a/.gitmodules b/.gitmodules index 6ad948c9a0a..f2520eb22ad 100644 --- a/.gitmodules +++ b/.gitmodules @@ -76,3 +76,6 @@ [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 diff --git a/CMakeLists.txt b/CMakeLists.txt index 9782bbf91a5..cf08ce4cfe6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -247,6 +247,7 @@ include (cmake/find_re2.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 diff --git a/cmake/find_h3.cmake b/cmake/find_h3.cmake new file mode 100644 index 00000000000..7f19157f978 --- /dev/null +++ b/cmake/find_h3.cmake @@ -0,0 +1,17 @@ +option (USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of bundled" ${NOT_UNBUNDLED}) + +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 geoCoord.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}") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 8d1d0ecd150..7861940412c 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -106,6 +106,11 @@ if (USE_INTERNAL_CPUID_LIBRARY) add_subdirectory (libcpuid) endif () +if (USE_INTERNAL_H3_LIBRARY) + add_subdirectory(h3) +endif () + + if (USE_INTERNAL_SSL_LIBRARY) if (NOT MAKE_STATIC_LIBRARIES) set (BUILD_SHARED 1) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 6b4cfab15c1..80eb62b86d0 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -19,6 +19,7 @@ target_link_libraries(clickhouse_functions ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash + m ${BASE64_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) @@ -60,3 +61,8 @@ if (USE_XXHASH) target_link_libraries(clickhouse_functions PRIVATE ${XXHASH_LIBRARY}) 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() diff --git a/dbms/src/Functions/geoToH3.cpp b/dbms/src/Functions/geoToH3.cpp new file mode 100644 index 00000000000..a4394e8940c --- /dev/null +++ b/dbms/src/Functions/geoToH3.cpp @@ -0,0 +1,171 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +extern "C" { +#include +} + +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"; + + FunctionGeoToH3(const Context & context) : context(context) {} + + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + + 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(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + int const_cnt = 0; + const auto size = input_rows_count; + + for (const auto idx : ext::range(0, 2)) + { + const auto column = block.getByPosition(arguments[idx]).column.get(); + if (typeid_cast(column)) + { + ++const_cnt; + } + else if (!typeid_cast *>(column)) + { + throw Exception( + "Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + } + } + + double resolution = 0; + bool is_const_resulution = false; + { + const auto column = block.getByPosition(arguments[2]).column.get(); + if (typeid_cast(column)) + { + is_const_resulution = true; + const auto col_const_res = static_cast(column); + resolution = col_const_res->getValue(); + } + else if (!typeid_cast *>(column)) + { + throw Exception( + "Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + } + else if (const_cnt == 2) + { + throw Exception( + "Illegal type " + column->getName() + " of arguments 3 of function " + getName() + + ". It must be const if arguments 1 and 2 are consts.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + + const auto col_lat = block.getByPosition(arguments[0]).column.get(); + const auto col_lon = block.getByPosition(arguments[1]).column.get(); + const auto col_res = block.getByPosition(arguments[2]).column.get(); + if (const_cnt == 0) + { + const auto col_vec_lat = static_cast *>(col_lat); + const auto col_vec_lon = static_cast *>(col_lon); + const auto col_vec_res = static_cast *>(col_res); + + auto dst = ColumnVector::create(); + auto & dst_data = dst->getData(); + dst_data.resize(size); + + for (const auto row : ext::range(0, size)) + { + const double lat = col_vec_lat->getData()[row]; + const double lon = col_vec_lon->getData()[row]; + if (!is_const_resulution) + { + resolution = col_vec_res->getData()[row]; + } + + GeoCoord coord; + setGeoDegs(&coord, lat, lon); + + H3Index hindex = H3_EXPORT(geoToH3)(&coord, resolution); + + dst_data[row] = hindex; + } + + block.getByPosition(result).column = std::move(dst); + } + else if (const_cnt == 2) + { + const auto col_const_lat = static_cast(col_lat); + const auto col_const_lon = static_cast(col_lon); + + const double lat = col_const_lat->getValue(); + const double lon = col_const_lon->getValue(); + + GeoCoord coord; + setGeoDegs(&coord, lat, lon); + H3Index hindex = H3_EXPORT(geoToH3)(&coord, resolution); + + block.getByPosition(result).column = DataTypeUInt64().createColumnConst(size, hindex); + } + else + { + throw Exception( + "Illegal types " + col_lat->getName() + ", " + col_lon->getName() + " of arguments 1, 2 of function " + getName() + + ". All must be either const or vector", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + +private: + const Context & context; +}; + + +void registerFunctionGeoToH3(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index 86d630260ec..6de25cf733c 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -42,6 +42,7 @@ void registerFunctionsGeo(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); void registerFunctionTransform(FunctionFactory &); +void registerFunctionGeoToH3(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -83,6 +84,7 @@ void registerFunctions() registerFunctionsNull(factory); registerFunctionsFindCluster(factory); registerFunctionTransform(factory); + registerFunctionGeoToH3(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl index 72572b775a5..28ae90ec139 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl @@ -133,7 +133,7 @@ sub main { split /[\s;,]+/, $ENV{SQL_FUZZY_FUNCTIONS} || file_read($ENV{SQL_FUZZY_FILE_FUNCTIONS} || 'clickhouse-functions') - || '__inner_restore_projection__ __inner_build_projection_composition__ convertCharset one_or_zero findClusterValue findClusterIndex toNullable coalesce isNotNull pointInEllipses transform pow acos asin tan cos tgamma lgamma erfc erf sqrt log10 exp10 e visitParamExtractFloat visitParamExtractUInt decodeURLComponent cutURLParameter cutQueryStringAndFragment cutFragment cutWWW URLPathHierarchy URLHierarchy extractURLParameterNames extractURLParameter queryStringAndFragment pathFull sin topLevelDomain domainWithoutWWW domain protocol greatCircleDistance extract match positionCaseInsensitiveUTF8 positionCaseInsensitive positionUTF8 position replaceRegexpAll replaceRegexpOne arrayStringConcat splitByString splitByChar alphaTokens endsWith startsWith appendTrailingCharIfAbsent substringUTF8 concatAssumeInjective reverseUTF8 upperUTF8 __inner_project__ upper lower length notEmpty trunc round roundAge roundDuration roundToExp2 reinterpretAsString reinterpretAsDateTime reinterpretAsDate reinterpretAsFloat64 reinterpretAsFloat32 reinterpretAsInt64 reinterpretAsInt8 reinterpretAsUInt32 toStartOfFiveMinute toISOYear toISOWeek concat toDecimal64 ifNull toStartOfDay toSecond addSeconds sleepEachRow materialize visitParamExtractInt toStartOfMinute toDayOfWeek toDayOfMonth bitShiftLeft emptyArrayUInt8 parseDateTimeBestEffort toTime toDateTimeOrNull toFloat32OrNull toInt16 IPv6NumToString atan substring arrayIntersect isInfinite toRelativeHourNum hex arrayEnumerateDense toUInt8OrZero toRelativeSecondNum toUInt64OrNull MACNumToString toInt32OrNull toDayOfYear toUnixTimestamp toString toDateOrZero subtractDays toMinute murmurHash3_64 murmurHash2_32 toUInt64 toUInt8 dictGetDateTime empty isFinite caseWithoutExpression caseWithoutExpr visitParamExtractRaw queryString dictGetInt32OrDefault caseWithExpression toInt8OrZero multiIf if intExp10 bitShiftRight less toUInt8OrNull toInt8OrNull bitmaskToArray toIntervalYear toFloat64OrZero dateDiff generateUUIDv4 arrayPopBack toIntervalMonth toUUID notEquals toInt16OrNull murmurHash2_64 hasAny toIntervalMinute isNull tupleElement replaceAll parseDateTimeBestEffortOrZero toFloat32OrZero lowerUTF8 notIn gcd like regionToPopulation MACStringToOUI notLike toStringCutToZero lcm parseDateTimeBestEffortOrNull not toInt32OrZero arrayFilter toInt16OrZero range equals now toTypeName toUInt32OrNull emptyArrayString dictGetDateTimeOrDefault bitRotateRight cutIPv6 toUInt32OrZero timezone reverse runningDifferenceStartingWithFirstValue toDateTime arrayPopFront toInt32 intHash64 extractURLParameters lowCardinalityIndices toStartOfMonth toYear hasAll rowNumberInAllBlocks bitTestAll arrayCount arraySort abs bitNot intDiv intDivOrZero firstSignificantSubdomain dictGetFloat32OrDefault reinterpretAsUInt16 toHour minus regionToArea unhex IPv4StringToNum toIntervalHour toInt8 dictGetFloat32 log IPv4NumToString modulo arrayEnumerate cutQueryString reinterpretAsFixedString countEqual bitTest toDecimal128 plus or reinterpretAsUInt64 toMonth visitParamExtractBool emptyArrayUInt64 replaceOne arrayReverseSort toFloat32 toRelativeMonthNum emptyArrayInt32 toRelativeYearNum arrayElement log2 array arrayReverse toUInt64OrZero emptyArrayFloat64 negate arrayPushBack subtractWeeks bitTestAny bitAnd toDecimal32 arrayPushFront lessOrEquals intExp2 toUInt16OrZero arrayConcat arrayCumSum arraySlice addDays dictGetUInt8 toUInt32 bitOr caseWithExpr toStartOfYear toIntervalDay MD5 emptyArrayUInt32 emptyArrayInt8 toMonday addMonths arrayUniq SHA256 arrayExists multiply toUInt16OrNull dictGetInt8 visitParamHas emptyArrayInt64 toIntervalSecond toDate sleep emptyArrayToSingle path toInt64OrZero SHA1 extractAll emptyArrayDate dumpColumnStructure toInt64 lengthUTF8 greatest arrayEnumerateUniq arrayDistinct arrayFirst toFixedString IPv4NumToStringClassC toFloat64OrNull IPv4ToIPv6 identity ceil toStartOfQuarter dictGetInt8OrDefault MACStringToNum emptyArrayUInt16 UUIDStringToNum dictGetUInt16 toStartOfFifteenMinutes toStartOfHour sumburConsistentHash toStartOfISOYear toRelativeQuarterNum toRelativeWeekNum toRelativeDayNum cbrt yesterday bitXor timeSlot timeSlots emptyArrayInt16 dictGetInt16 toYYYYMM toYYYYMMDDhhmmss toUInt16 addMinutes addHours addWeeks nullIf subtractSeconds subtractMinutes toIntervalWeek subtractHours isNaN subtractMonths toDateOrNull subtractYears toTimeZone formatDateTime has cityHash64 intHash32 fragment regionToCity indexOf regionToDistrict regionToCountry visibleWidth regionToContinent regionToTopContinent toColumnTypeName regionHierarchy CHAR_LENGTH least divide SEHierarchy dictGetDate OSToRoot SEToRoot OSIn SEIn regionToName dictGetStringOrDefault OSHierarchy exp floor dictGetUInt8OrDefault dictHas dictGetUInt64 cutToFirstSignificantSubdomain dictGetInt32 pointInPolygon dictGetInt64 blockNumber IPv6StringToNum dictGetString dictGetFloat64 dictGetUUID CHARACTER_LENGTH toQuarter dictGetHierarchy toFloat64 arraySum toInt64OrNull dictIsIn dictGetUInt16OrDefault dictGetUInt32OrDefault emptyArrayDateTime greater jumpConsistentHash dictGetUInt64OrDefault dictGetInt16OrDefault dictGetInt64OrDefault reinterpretAsInt32 dictGetUInt32 murmurHash3_32 bar dictGetUUIDOrDefault rand modelEvaluate arrayReduce farmHash64 bitmaskToList formatReadableSize halfMD5 SHA224 arrayMap sipHash64 dictGetFloat64OrDefault sipHash128 metroHash64 murmurHash3_128 yandexConsistentHash emptyArrayFloat32 arrayAll toYYYYMMDD today arrayFirstIndex greaterOrEquals arrayDifference visitParamExtractString toDateTimeOrZero globalNotIn throwIf and xor currentDatabase hostName URLHash getSizeOfEnumType defaultValueOfArgumentType blockSize tuple arrayCumSumNonNegative rowNumberInBlock arrayResize ignore toRelativeMinuteNum indexHint reinterpretAsInt16 addYears arrayJoin replicate hasColumnInTable version regionIn uptime runningAccumulate runningDifference assumeNotNull pi finalizeAggregation toLowCardinality exp2 lowCardinalityKeys in globalIn dictGetDateOrDefault rand64 CAST bitRotateLeft randConstant UUIDNumToString reinterpretAsUInt8 truncate ceiling retention maxIntersections groupBitXor groupBitOr uniqUpTo uniqCombined uniqExact uniq covarPop stddevPop varPop covarSamp varSamp sumMap corrStable corr quantileTiming quantileDeterministic quantilesExact uniqHLL12 quantilesTiming covarPopStable stddevSampStable quantilesExactWeighted quantileExactWeighted quantileTimingWeighted quantileExact quantilesDeterministic quantiles topK sumWithOverflow count groupArray stddevSamp groupArrayInsertAt quantile quantilesTimingWeighted quantileTDigest quantilesTDigest windowFunnel min argMax varSampStable maxIntersectionsPosition quantilesTDigestWeighted groupUniqArray sequenceCount sumKahan any anyHeavy histogram quantileTDigestWeighted max groupBitAnd argMin varPopStable avg sequenceMatch stddevPopStable sum anyLast covarSampStable BIT_XOR medianExactWeighted medianTiming medianExact median medianDeterministic VAR_SAMP STDDEV_POP medianTDigest VAR_POP medianTDigestWeighted BIT_OR STDDEV_SAMP medianTimingWeighted COVAR_SAMP COVAR_POP BIT_AND' + || '__inner_restore_projection__ __inner_build_projection_composition__ convertCharset one_or_zero findClusterValue findClusterIndex toNullable coalesce isNotNull pointInEllipses geoToH3 transform pow acos asin tan cos tgamma lgamma erfc erf sqrt log10 exp10 e visitParamExtractFloat visitParamExtractUInt decodeURLComponent cutURLParameter cutQueryStringAndFragment cutFragment cutWWW URLPathHierarchy URLHierarchy extractURLParameterNames extractURLParameter queryStringAndFragment pathFull sin topLevelDomain domainWithoutWWW domain protocol greatCircleDistance extract match positionCaseInsensitiveUTF8 positionCaseInsensitive positionUTF8 position replaceRegexpAll replaceRegexpOne arrayStringConcat splitByString splitByChar alphaTokens endsWith startsWith appendTrailingCharIfAbsent substringUTF8 concatAssumeInjective reverseUTF8 upperUTF8 __inner_project__ upper lower length notEmpty trunc round roundAge roundDuration roundToExp2 reinterpretAsString reinterpretAsDateTime reinterpretAsDate reinterpretAsFloat64 reinterpretAsFloat32 reinterpretAsInt64 reinterpretAsInt8 reinterpretAsUInt32 toStartOfFiveMinute toISOYear toISOWeek concat toDecimal64 ifNull toStartOfDay toSecond addSeconds sleepEachRow materialize visitParamExtractInt toStartOfMinute toDayOfWeek toDayOfMonth bitShiftLeft emptyArrayUInt8 parseDateTimeBestEffort toTime toDateTimeOrNull toFloat32OrNull toInt16 IPv6NumToString atan substring arrayIntersect isInfinite toRelativeHourNum hex arrayEnumerateDense toUInt8OrZero toRelativeSecondNum toUInt64OrNull MACNumToString toInt32OrNull toDayOfYear toUnixTimestamp toString toDateOrZero subtractDays toMinute murmurHash3_64 murmurHash2_32 toUInt64 toUInt8 dictGetDateTime empty isFinite caseWithoutExpression caseWithoutExpr visitParamExtractRaw queryString dictGetInt32OrDefault caseWithExpression toInt8OrZero multiIf if intExp10 bitShiftRight less toUInt8OrNull toInt8OrNull bitmaskToArray toIntervalYear toFloat64OrZero dateDiff generateUUIDv4 arrayPopBack toIntervalMonth toUUID notEquals toInt16OrNull murmurHash2_64 hasAny toIntervalMinute isNull tupleElement replaceAll parseDateTimeBestEffortOrZero toFloat32OrZero lowerUTF8 notIn gcd like regionToPopulation MACStringToOUI notLike toStringCutToZero lcm parseDateTimeBestEffortOrNull not toInt32OrZero arrayFilter toInt16OrZero range equals now toTypeName toUInt32OrNull emptyArrayString dictGetDateTimeOrDefault bitRotateRight cutIPv6 toUInt32OrZero timezone reverse runningDifferenceStartingWithFirstValue toDateTime arrayPopFront toInt32 intHash64 extractURLParameters lowCardinalityIndices toStartOfMonth toYear hasAll rowNumberInAllBlocks bitTestAll arrayCount arraySort abs bitNot intDiv intDivOrZero firstSignificantSubdomain dictGetFloat32OrDefault reinterpretAsUInt16 toHour minus regionToArea unhex IPv4StringToNum toIntervalHour toInt8 dictGetFloat32 log IPv4NumToString modulo arrayEnumerate cutQueryString reinterpretAsFixedString countEqual bitTest toDecimal128 plus or reinterpretAsUInt64 toMonth visitParamExtractBool emptyArrayUInt64 replaceOne arrayReverseSort toFloat32 toRelativeMonthNum emptyArrayInt32 toRelativeYearNum arrayElement log2 array arrayReverse toUInt64OrZero emptyArrayFloat64 negate arrayPushBack subtractWeeks bitTestAny bitAnd toDecimal32 arrayPushFront lessOrEquals intExp2 toUInt16OrZero arrayConcat arrayCumSum arraySlice addDays dictGetUInt8 toUInt32 bitOr caseWithExpr toStartOfYear toIntervalDay MD5 emptyArrayUInt32 emptyArrayInt8 toMonday addMonths arrayUniq SHA256 arrayExists multiply toUInt16OrNull dictGetInt8 visitParamHas emptyArrayInt64 toIntervalSecond toDate sleep emptyArrayToSingle path toInt64OrZero SHA1 extractAll emptyArrayDate dumpColumnStructure toInt64 lengthUTF8 greatest arrayEnumerateUniq arrayDistinct arrayFirst toFixedString IPv4NumToStringClassC toFloat64OrNull IPv4ToIPv6 identity ceil toStartOfQuarter dictGetInt8OrDefault MACStringToNum emptyArrayUInt16 UUIDStringToNum dictGetUInt16 toStartOfFifteenMinutes toStartOfHour sumburConsistentHash toStartOfISOYear toRelativeQuarterNum toRelativeWeekNum toRelativeDayNum cbrt yesterday bitXor timeSlot timeSlots emptyArrayInt16 dictGetInt16 toYYYYMM toYYYYMMDDhhmmss toUInt16 addMinutes addHours addWeeks nullIf subtractSeconds subtractMinutes toIntervalWeek subtractHours isNaN subtractMonths toDateOrNull subtractYears toTimeZone formatDateTime has cityHash64 intHash32 fragment regionToCity indexOf regionToDistrict regionToCountry visibleWidth regionToContinent regionToTopContinent toColumnTypeName regionHierarchy CHAR_LENGTH least divide SEHierarchy dictGetDate OSToRoot SEToRoot OSIn SEIn regionToName dictGetStringOrDefault OSHierarchy exp floor dictGetUInt8OrDefault dictHas dictGetUInt64 cutToFirstSignificantSubdomain dictGetInt32 pointInPolygon dictGetInt64 blockNumber IPv6StringToNum dictGetString dictGetFloat64 dictGetUUID CHARACTER_LENGTH toQuarter dictGetHierarchy toFloat64 arraySum toInt64OrNull dictIsIn dictGetUInt16OrDefault dictGetUInt32OrDefault emptyArrayDateTime greater jumpConsistentHash dictGetUInt64OrDefault dictGetInt16OrDefault dictGetInt64OrDefault reinterpretAsInt32 dictGetUInt32 murmurHash3_32 bar dictGetUUIDOrDefault rand modelEvaluate arrayReduce farmHash64 bitmaskToList formatReadableSize halfMD5 SHA224 arrayMap sipHash64 dictGetFloat64OrDefault sipHash128 metroHash64 murmurHash3_128 yandexConsistentHash emptyArrayFloat32 arrayAll toYYYYMMDD today arrayFirstIndex greaterOrEquals arrayDifference visitParamExtractString toDateTimeOrZero globalNotIn throwIf and xor currentDatabase hostName URLHash getSizeOfEnumType defaultValueOfArgumentType blockSize tuple arrayCumSumNonNegative rowNumberInBlock arrayResize ignore toRelativeMinuteNum indexHint reinterpretAsInt16 addYears arrayJoin replicate hasColumnInTable version regionIn uptime runningAccumulate runningDifference assumeNotNull pi finalizeAggregation toLowCardinality exp2 lowCardinalityKeys in globalIn dictGetDateOrDefault rand64 CAST bitRotateLeft randConstant UUIDNumToString reinterpretAsUInt8 truncate ceiling retention maxIntersections groupBitXor groupBitOr uniqUpTo uniqCombined uniqExact uniq covarPop stddevPop varPop covarSamp varSamp sumMap corrStable corr quantileTiming quantileDeterministic quantilesExact uniqHLL12 quantilesTiming covarPopStable stddevSampStable quantilesExactWeighted quantileExactWeighted quantileTimingWeighted quantileExact quantilesDeterministic quantiles topK sumWithOverflow count groupArray stddevSamp groupArrayInsertAt quantile quantilesTimingWeighted quantileTDigest quantilesTDigest windowFunnel min argMax varSampStable maxIntersectionsPosition quantilesTDigestWeighted groupUniqArray sequenceCount sumKahan any anyHeavy histogram quantileTDigestWeighted max groupBitAnd argMin varPopStable avg sequenceMatch stddevPopStable sum anyLast covarSampStable BIT_XOR medianExactWeighted medianTiming medianExact median medianDeterministic VAR_SAMP STDDEV_POP medianTDigest VAR_POP medianTDigestWeighted BIT_OR STDDEV_SAMP medianTimingWeighted COVAR_SAMP COVAR_POP BIT_AND' ]; # $functions = [grep { not $_ ~~ [qw( )] } @$functions]; # will be removed # select name from system.table_functions format TSV; diff --git a/dbms/tests/queries/0_stateless/00926_geo_to_h3.reference b/dbms/tests/queries/0_stateless/00926_geo_to_h3.reference new file mode 100644 index 00000000000..ad594f0e81f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00926_geo_to_h3.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql b/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql new file mode 100644 index 00000000000..38a60c0061e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql @@ -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(55.77922738, 37.63098076, 15); +select geoToH3(lat, lon, resolution) from table1 order by lat, lon, resolution; +select geoToH3(lat, lon, 15) from table1 order by lat, lon, geoToH3(lat, lon, 15); +select lat, lon, geoToH3(lat, lon, 15) from table1 order by lat, lon, geoToH3(lat, lon, 15); +select geoToH3(lat, lon, resolution), count(*) from table1 group by geoToH3(lat, lon, resolution) order by geoToH3(lat, lon, resolution); + +DROP TABLE table1 diff --git a/docs/ru/query_language/functions/geo.md b/docs/ru/query_language/functions/geo.md index 4423a167e2e..ec1033eb49b 100644 --- a/docs/ru/query_language/functions/geo.md +++ b/docs/ru/query_language/functions/geo.md @@ -99,4 +99,37 @@ SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res └─────┘ ``` +## geoToH3 + +Получает H3 индекс точки (lat, lon) с заданным разрешением + +``` +pointInPolygon(lat, lon, resolution) +``` + +**Входные значения** + +- `lat` - географическая широта. Тип данных — [Float64](../../data_types/float.md). +- `lon` - географическая долгота. Тип данных — [Float64](../../data_types/float.md). +- `resolution` - требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значение — `[0, 15]`. + +Параметры `lat` и `lon` должны быть одновременно или константными, или нет. Если параметры `lat` и `lon` не являются константными, то параметр `resolution` не может быть константным. + +**Возвращаемые значения** + +Возвращает значение с типом [UInt64] (../../data_types/int_uint.md). +`0` в случае ошибки. +Иначе возвращается индексный номер шестиугольника. + +**Пример** + +``` sql +SELECT geoToH3(55.71290588, 37.79506683, 15) as h3Index +``` +``` +┌────────────h3Index─┐ +│ 644325524701193974 │ +└────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/geo/)