mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #25595 from ClickHouse/revert-25593-revert-24867-ncb/h3-indexing-funcs
Implement h3ToGeo function [2]
This commit is contained in:
commit
f8b4b3e81b
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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>();
|
||||
}
|
||||
|
@ -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>();
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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>();
|
||||
}
|
||||
|
@ -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>();
|
||||
}
|
||||
|
@ -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());
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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
96
src/Functions/h3toGeo.cpp
Normal 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
|
@ -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);
|
||||
|
@ -300,6 +300,7 @@ SRCS(
|
||||
h3ToParent.cpp
|
||||
h3ToString.cpp
|
||||
h3kRing.cpp
|
||||
h3toGeo.cpp
|
||||
hasColumnInTable.cpp
|
||||
hasThreadFuzzer.cpp
|
||||
hasToken.cpp
|
||||
|
@ -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;
|
||||
|
32
tests/queries/0_stateless/01906_h3_to_geo.reference
Normal file
32
tests/queries/0_stateless/01906_h3_to_geo.reference
Normal 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
|
61
tests/queries/0_stateless/01906_h3_to_geo.sql
Normal file
61
tests/queries/0_stateless/01906_h3_to_geo.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user