Merge pull request #25595 from ClickHouse/revert-25593-revert-24867-ncb/h3-indexing-funcs

Implement h3ToGeo function [2]
This commit is contained in:
Nikita Mikhaylov 2021-07-13 20:08:17 +03:00 committed by GitHub
commit f8b4b3e81b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 352 additions and 48 deletions

View File

@ -195,6 +195,41 @@ Result:
└────────────────────┘
```
## h3ToGeo {#h3togeo}
Returns `(lon, lat)` that corresponds to the provided H3 index.
**Syntax**
``` sql
h3ToGeo(h3Index)
```
**Arguments**
- `h3Index` — H3 Index. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned values**
- `lon` — Longitude. Type: [Float64](../../../sql-reference/data-types/float.md).
- `lat` — Latitude. Type: [Float64](../../../sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT h3ToGeo(644325524701193974) coordinates;
```
Result:
``` text
┌─coordinates───────────────────────────┐
│ (37.79506616830252,55.71290243145668) │
└───────────────────────────────────────┘
```
## h3kRing {#h3kring}
Lists all the [H3](#h3index) hexagons in the raduis of `k` from the given hexagon in random order.

View File

@ -1,4 +1,7 @@
#include "Connection.h"
#if USE_LIBPQXX
#include <common/logger_useful.h>
namespace postgres
@ -72,3 +75,5 @@ void Connection::connect()
updateConnection();
}
}
#endif

View File

@ -1,5 +1,11 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include "config_core.h"
#endif
#if USE_LIBPQXX
#include <pqxx/pqxx> // Y_IGNORE
#include <Core/Types.h>
#include <boost/noncopyable.hpp>
@ -45,3 +51,5 @@ private:
Poco::Logger * log;
};
}
#endif

View File

@ -1,5 +1,11 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include "config_core.h"
#endif
#if USE_LIBPQXX
#include <pqxx/pqxx> // Y_IGNORE
#include <Core/Types.h>
#include <common/BorrowedObjectPool.h>
@ -35,3 +41,5 @@ private:
using ConnectionHolderPtr = std::unique_ptr<ConnectionHolder>;
}
#endif

View File

@ -1,4 +1,7 @@
#include "PoolWithFailover.h"
#if USE_LIBPQXX
#include "Utils.h"
#include <Common/parseRemoteDescription.h>
#include <Common/Exception.h>
@ -136,3 +139,5 @@ ConnectionHolderPtr PoolWithFailover::get()
throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, "Unable to connect to any of the replicas");
}
}
#endif

View File

@ -1,5 +1,12 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include "config_core.h"
#endif
#if USE_LIBPQXX
#include "ConnectionHolder.h"
#include <mutex>
#include <Poco/Util/AbstractConfiguration.h>
@ -63,3 +70,5 @@ private:
using PoolWithFailoverPtr = std::shared_ptr<PoolWithFailover>;
}
#endif

View File

@ -1,4 +1,7 @@
#include "Utils.h"
#if USE_LIBPQXX
#include <IO/Operators.h>
namespace postgres
@ -17,3 +20,5 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S
}
}
#endif

View File

@ -1,5 +1,11 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include "config_core.h"
#endif
#if USE_LIBPQXX
#include <pqxx/pqxx> // Y_IGNORE
#include <Core/Types.h>
#include "Connection.h"
@ -15,3 +21,5 @@ namespace postgres
{
ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password);
}
#endif

View File

@ -31,6 +31,10 @@ SRCS(
MySQL/PacketsProtocolText.cpp
MySQL/PacketsReplication.cpp
NamesAndTypes.cpp
PostgreSQL/Connection.cpp
PostgreSQL/PoolWithFailover.cpp
PostgreSQL/Utils.cpp
PostgreSQL/insertPostgreSQLValue.cpp
PostgreSQLProtocol.cpp
QueryProcessingStage.cpp
Settings.cpp

View File

@ -44,8 +44,9 @@ public:
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt8",
arg->getName(), 1, getName());
return std::make_shared<DataTypeFloat64>();
}
@ -62,8 +63,10 @@ public:
{
const int resolution = col_hindex->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is ",
resolution, getName(), MAX_H3_RES);
// Numerical constant is 180 degrees / pi / Earth radius, Earth radius is from h3 sources
Float64 res = 8.99320592271288084e-6 * getHexagonEdgeLengthAvgM(resolution);

View File

@ -49,8 +49,9 @@ public:
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt8",
arg->getName(), 1, getName());
return std::make_shared<DataTypeFloat64>();
}
@ -67,8 +68,10 @@ public:
{
const UInt64 resolution = col_hindex->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is ",
resolution, getName(), MAX_H3_RES);
Float64 res = getHexagonEdgeLengthAvgM(resolution);

View File

@ -41,8 +41,9 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeUInt8>();
}

View File

@ -41,8 +41,9 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeUInt8>();
}

View File

@ -44,8 +44,9 @@ public:
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt8",
arg->getName(), 1, getName());
return std::make_shared<DataTypeFloat64>();
}
@ -62,8 +63,10 @@ public:
{
const UInt64 resolution = col_hindex->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is ",
resolution, getName(), MAX_H3_RES);
Float64 res = getHexagonAreaAvgM2(resolution);

View File

@ -41,14 +41,16 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 2, getName());
return std::make_shared<DataTypeUInt8>();
}

View File

@ -41,8 +41,9 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeUInt8>();
}

View File

@ -50,14 +50,16 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt8",
arg->getName(), 2, getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
@ -81,14 +83,17 @@ public:
const UInt8 child_resolution = col_resolution->getUInt(row);
if (child_resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(child_resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is {}",
toString(child_resolution), getName(), toString(MAX_H3_RES));
const size_t vec_size = cellToChildrenSize(parent_hindex, child_resolution);
if (vec_size > MAX_ARRAY_SIZE)
throw Exception("The result of function" + getName()
+ " (array of " + toString(vec_size) + " elements) will be too large with resolution argument = "
+ toString(child_resolution), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
throw Exception(
ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"The result of function {} (array of {} elements) will be too large with resolution argument = {}",
getName(), toString(vec_size), toString(child_resolution));
hindex_vec.resize(vec_size);
cellToChildren(parent_hindex, child_resolution, hindex_vec.data());

View File

@ -44,14 +44,16 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!WhichDataType(arg).isUInt8())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt8",
arg->getName(), 2, getName());
return std::make_shared<DataTypeUInt64>();
}
@ -71,8 +73,10 @@ public:
const UInt8 resolution = col_resolution->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"The argument 'resolution' ({}) of function {} is out of bounds because the maximum resolution in H3 library is {}",
toString(resolution), getName(), toString(MAX_H3_RES));
UInt64 res = cellToParent(hindex, resolution);

View File

@ -42,8 +42,9 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeString>();
}
@ -67,16 +68,14 @@ public:
const UInt64 hindex = col_hindex->getUInt(i);
if (!isValidCell(hindex))
{
throw Exception("Invalid H3 index: " + std::to_string(hindex), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", hindex);
h3ToString(hindex, pos, H3_INDEX_STRING_LENGTH);
// move to end of the index
while (*pos != '\0')
{
pos++;
}
vec_offsets[i] = ++pos - begin;
}
vec_res.resize(pos - begin);

View File

@ -47,14 +47,16 @@ public:
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 UInt64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!isInteger(arg))
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be integer",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be integer",
arg->getName(), 2, getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}

96
src/Functions/h3toGeo.cpp Normal file
View File

@ -0,0 +1,96 @@
#if !defined(ARCADIA_BUILD)
# include "config_functions.h"
#endif
#if USE_H3
#include <array>
#include <math.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
/// Implements the function h3ToGeo which takes a single argument (h3Index)
/// and returns the longitude and latitude that correspond to the provided h3 index
class FunctionH3ToGeo : public IFunction
{
public:
static constexpr auto name = "h3ToGeo";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3ToGeo>(); }
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
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeFloat64>(), std::make_shared<DataTypeFloat64>()},
Strings{"longitude", "latitude"});
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_index = arguments[0].column.get();
auto latitude = ColumnFloat64::create(input_rows_count);
auto longitude = ColumnFloat64::create(input_rows_count);
ColumnFloat64::Container & lon_data = longitude->getData();
ColumnFloat64::Container & lat_data = latitude->getData();
for (size_t row = 0; row < input_rows_count; ++row)
{
H3Index h3index = col_index->getUInt(row);
LatLng coord{};
cellToLatLng(h3index,&coord);
lon_data[row] = radsToDegs(coord.lng);
lat_data[row] = radsToDegs(coord.lat);
}
MutableColumns columns;
columns.emplace_back(std::move(longitude));
columns.emplace_back(std::move(latitude));
return ColumnTuple::create(std::move(columns));
}
};
}
void registerFunctionH3ToGeo(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3ToGeo>();
}
}
#endif

View File

@ -28,6 +28,7 @@ void registerFunctionSvg(FunctionFactory & factory);
#if USE_H3
void registerFunctionGeoToH3(FunctionFactory &);
void registerFunctionH3ToGeo(FunctionFactory &);
void registerFunctionH3EdgeAngle(FunctionFactory &);
void registerFunctionH3EdgeLengthM(FunctionFactory &);
void registerFunctionH3GetResolution(FunctionFactory &);
@ -66,6 +67,7 @@ void registerFunctionsGeo(FunctionFactory & factory)
#if USE_H3
registerFunctionGeoToH3(factory);
registerFunctionH3ToGeo(factory);
registerFunctionH3EdgeAngle(factory);
registerFunctionH3EdgeLengthM(factory);
registerFunctionH3GetResolution(factory);

View File

@ -300,6 +300,7 @@ SRCS(
h3ToParent.cpp
h3ToString.cpp
h3kRing.cpp
h3toGeo.cpp
hasColumnInTable.cpp
hasThreadFuzzer.cpp
hasToken.cpp

View File

@ -7,3 +7,4 @@ SELECT h3kRing(0xFFFFFFFFFFFFFF, 1000) FORMAT Null;
SELECT h3GetBaseCell(0xFFFFFFFFFFFFFF) FORMAT Null;
SELECT h3GetResolution(0xFFFFFFFFFFFFFF) FORMAT Null;
SELECT h3kRing(0xFFFFFFFFFFFFFF, 10) FORMAT Null;
SELECT h3ToGeo(0xFFFFFFFFFFFFFF) FORMAT Null;

View File

@ -0,0 +1,32 @@
(-173.6412167681162,-14.130272474941535)
(59.48137613600854,58.020407687755686)
(172.68095885060296,-83.6576608516349)
(-94.46556851304558,-69.1999982492279)
(-8.188263637093279,-55.856179102736284)
(77.25594891852249,47.39278564360122)
(135.11348004704536,36.60778126579667)
(39.28534828967223,49.07710003066973)
(124.71163478198051,-27.481172161567258)
(-147.4887686066785,76.73237945824442)
(86.63291906118863,-25.52526285188784)
(23.27751790712118,13.126101362212724)
(-70.40163237204142,-63.12562536833242)
(15.642428355535966,40.285813505163574)
(-76.53411447979884,54.5560449693637)
(8.19906334981474,67.69370966550179)
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok
ok

View File

@ -0,0 +1,61 @@
DROP TABLE IF EXISTS h3_indexes;
CREATE TABLE h3_indexes (h3_index UInt64) ENGINE = Memory;
-- Random geo coordinates were generated using the H3 tool: https://github.com/ClickHouse-Extras/h3/blob/master/src/apps/testapps/mkRandGeo.c at various resolutions from 0 to 15.
-- Corresponding H3 index values were in turn generated with those geo coordinates using `geoToH3(lon, lat, res)` ClickHouse function for the following test.
INSERT INTO h3_indexes VALUES (579205133326352383);
INSERT INTO h3_indexes VALUES (581263419093549055);
INSERT INTO h3_indexes VALUES (589753847883235327);
INSERT INTO h3_indexes VALUES (594082350283882495);
INSERT INTO h3_indexes VALUES (598372386957426687);
INSERT INTO h3_indexes VALUES (599542359671177215);
INSERT INTO h3_indexes VALUES (604296355086598143);
INSERT INTO h3_indexes VALUES (608785214872748031);
INSERT INTO h3_indexes VALUES (615732192485572607);
INSERT INTO h3_indexes VALUES (617056794467368959);
INSERT INTO h3_indexes VALUES (624586477873168383);
INSERT INTO h3_indexes VALUES (627882919484481535);
INSERT INTO h3_indexes VALUES (634600058503392255);
INSERT INTO h3_indexes VALUES (635544851677385791);
INSERT INTO h3_indexes VALUES (639763125756281263);
INSERT INTO h3_indexes VALUES (644178757620501158);
SELECT h3ToGeo(h3_index) FROM h3_indexes ORDER BY h3_index;
DROP TABLE h3_indexes;
DROP TABLE IF EXISTS h3_geo;
-- compare if the results of h3ToGeo and geoToH3 are the same
CREATE TABLE h3_geo(lat Float64, lon Float64, res UInt8) ENGINE = Memory;
INSERT INTO h3_geo VALUES (-173.6412167681162, -14.130272474941535, 0);
INSERT INTO h3_geo VALUES (59.48137613600854, 58.020407687755686, 1);
INSERT INTO h3_geo VALUES (172.68095885060296, -83.6576608516349, 2);
INSERT INTO h3_geo VALUES (-94.46556851304558, -69.1999982492279, 3);
INSERT INTO h3_geo VALUES (-8.188263637093279, -55.856179102736284, 4);
INSERT INTO h3_geo VALUES (77.25594891852249, 47.39278564360122, 5);
INSERT INTO h3_geo VALUES (135.11348004704536, 36.60778126579667, 6);
INSERT INTO h3_geo VALUES (39.28534828967223, 49.07710003066973, 7);
INSERT INTO h3_geo VALUES (124.71163478198051, -27.481172161567258, 8);
INSERT INTO h3_geo VALUES (-147.4887686066785, 76.73237945824442, 9);
INSERT INTO h3_geo VALUES (86.63291906118863, -25.52526285188784, 10);
INSERT INTO h3_geo VALUES (23.27751790712118, 13.126101362212724, 11);
INSERT INTO h3_geo VALUES (-70.40163237204142, -63.12562536833242, 12);
INSERT INTO h3_geo VALUES (15.642428355535966, 40.285813505163574, 13);
INSERT INTO h3_geo VALUES (-76.53411447979884, 54.5560449693637, 14);
INSERT INTO h3_geo VALUES (8.19906334981474, 67.69370966550179, 15);
SELECT result FROM (
SELECT
(lat, lon) AS input_geo,
h3ToGeo(geoToH3(lat, lon, res)) AS output_geo,
if(input_geo = output_geo, 'ok', 'fail') AS result
FROM h3_geo
);
DROP TABLE h3_geo;