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

This commit is contained in:
Alexey Milovidov 2019-12-16 21:38:03 +03:00
commit 63884d1b1a
17 changed files with 636 additions and 0 deletions

View File

@ -0,0 +1,78 @@
#include "config_functions.h"
#if USE_H3
# 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
{
class FunctionH3EdgeAngle : public IFunction
{
public:
static constexpr auto name = "h3EdgeAngle";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3EdgeAngle>(); }
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
{
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);
return std::make_shared<DataTypeFloat64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const int resolution = col_hindex->getUInt(row);
// Numerical constant is 180 degrees / pi / Earth radius, Earth radius is from h3 sources
Float64 res = 8.99320592271288084e-6 * H3_EXPORT(edgeLengthM)(resolution);
dst_data[row] = res;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionH3EdgeAngle(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3EdgeAngle>();
}
}
#endif

View File

@ -0,0 +1,82 @@
#include "config_functions.h"
#if USE_H3
# 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
{
// Average metric edge length of H3 hexagon. The edge length `e` for given resolution `res` can
// be used for converting metric search radius `radius` to hexagon search ring size `k` that is
// used by `H3kRing` function. For small enough search area simple flat approximation can be used,
// i.e. the smallest `k` that satisfies relation `3 k^2 - 3 k + 1 >= (radius / e)^2` should be
// chosen
class FunctionH3EdgeLengthM : public IFunction
{
public:
static constexpr auto name = "h3EdgeLengthM";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3EdgeLengthM>(); }
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
{
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);
return std::make_shared<DataTypeFloat64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const int resolution = col_hindex->getUInt(row);
Float64 res = H3_EXPORT(edgeLengthM)(resolution);
dst_data[row] = res;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionH3EdgeLengthM(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3EdgeLengthM>();
}
}
#endif

View File

@ -0,0 +1,77 @@
#include "config_functions.h"
#if USE_H3
# 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
{
class FunctionH3GetResolution : public IFunction
{
public:
static constexpr auto name = "h3GetResolution";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3GetResolution>(); }
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
{
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);
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const UInt64 hindex = col_hindex->getUInt(row);
UInt8 res = H3_EXPORT(h3GetResolution)(hindex);
dst_data[row] = res;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionH3GetResolution(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetResolution>();
}
}
#endif

View File

@ -0,0 +1,77 @@
#include "config_functions.h"
#if USE_H3
# 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
{
class FunctionH3IsValid : public IFunction
{
public:
static constexpr auto name = "h3IsValid";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3IsValid>(); }
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
{
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);
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (const auto row : ext::range(0, input_rows_count))
{
const UInt64 hindex = col_hindex->getUInt(row);
UInt8 is_valid = H3_EXPORT(h3IsValid)(hindex) == 0 ? 0 : 1;
dst_data[row] = is_valid;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionH3IsValid(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3IsValid>();
}
}
#endif

View File

@ -0,0 +1,103 @@
#include "config_functions.h"
#if USE_H3
# include <vector>
# include <Columns/ColumnArray.h>
# include <Columns/ColumnsNumber.h>
# include <DataTypes/DataTypeArray.h>
# include <DataTypes/DataTypesNumber.h>
# include <DataTypes/IDataType.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
{
class FunctionH3KRing : public IFunction
{
public:
static constexpr auto name = "h3kRing";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3KRing>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
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);
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);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
const auto col_k = block.getByPosition(arguments[1]).column.get();
auto dst = ColumnArray::create(ColumnUInt64::create());
auto & dst_data = dst->getData();
auto & dst_offsets = dst->getOffsets();
dst_offsets.resize(input_rows_count);
auto current_offset = 0;
std::vector<H3Index> hindex_vec;
for (const auto row : ext::range(0, input_rows_count))
{
const H3Index origin_hindex = col_hindex->getUInt(row);
const int k = col_k->getInt(row);
const auto vec_size = H3_EXPORT(maxKringSize)(k);
hindex_vec.resize(vec_size);
H3_EXPORT(kRing)(origin_hindex, k, hindex_vec.data());
dst_data.reserve(dst_data.size() + vec_size);
for (auto hindex : hindex_vec)
{
if (hindex != 0)
{
++current_offset;
dst_data.insert(hindex);
}
}
dst_offsets[row] = current_offset;
}
block.getByPosition(result).column = std::move(dst);
}
};
void registerFunctionH3KRing(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3KRing>();
}
}
#endif

View File

@ -14,6 +14,11 @@ void registerFunctionGeohashesInBox(FunctionFactory & factory);
#if USE_H3
void registerFunctionGeoToH3(FunctionFactory &);
void registerFunctionH3EdgeAngle(FunctionFactory &);
void registerFunctionH3EdgeLengthM(FunctionFactory &);
void registerFunctionH3GetResolution(FunctionFactory &);
void registerFunctionH3IsValid(FunctionFactory &);
void registerFunctionH3KRing(FunctionFactory &);
#endif
void registerFunctionsGeo(FunctionFactory & factory)
@ -27,6 +32,11 @@ void registerFunctionsGeo(FunctionFactory & factory)
#if USE_H3
registerFunctionGeoToH3(factory);
registerFunctionH3EdgeAngle(factory);
registerFunctionH3EdgeLengthM(factory);
registerFunctionH3GetResolution(factory);
registerFunctionH3IsValid(factory);
registerFunctionH3KRing(factory);
#endif
}

View File

@ -0,0 +1,3 @@
1
10
15

View File

@ -0,0 +1,3 @@
SELECT h3GetResolution(581276613233082367);
SELECT h3GetResolution(621807531097128959);
SELECT h3GetResolution(644325529233966508);

View File

@ -0,0 +1,4 @@
1
1
0
0

View File

@ -0,0 +1,4 @@
SELECT h3IsValid(581276613233082367);
SELECT h3IsValid(621807531097128959);
SELECT h3IsValid(Cast(0, 'UInt64'));
SELECT h3IsValid(100000000000000000);

View File

@ -0,0 +1,3 @@
[581250224954015743,581259021047037951,581267817140060159,581276613233082367,581500913605148671,581518505791193087,581764796395814911]
[581276613233082367]
[581276613233082367]

View File

@ -0,0 +1,3 @@
SELECT arraySort(h3kRing(581276613233082367, 1));
SELECT h3kRing(581276613233082367, 0);
SELECT h3kRing(581276613233082367, -1);

View File

@ -0,0 +1,2 @@
418676.0055
22606.3794

View File

@ -0,0 +1,2 @@
SELECT h3EdgeLengthM(1);
SELECT h3EdgeLengthM(4);

View File

@ -0,0 +1,2 @@
0.0005927224846720883
0

View File

@ -0,0 +1,2 @@
SELECT h3EdgeAngle(10);
SELECT h3EdgeLengthM(2) * 180 / pi() / 6371007.180918475 - h3EdgeAngle(2);

View File

@ -35,6 +35,38 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)
└───────────────────────────────────────────────────────────────────┘
```
## greatCircleAngle
Вычисляет угловое расстояние на сфере по [формуле большого круга](https://en.wikipedia.org/wiki/Great-circle_distance).
```sql
greatCircleDistance(lon1Deg, lat1Deg, lon2Deg, lat2Deg)
```
**Входные параметры**
- `lon1Deg` — долгота первой точки в градусах.
- `lat1Deg` — широта первой точки в градусах.
- `lon2Deg` — долгота второй точки в градусах.
- `lat2Deg` — широта второй точки в градусах.
**Возвращаемое значение**
Длина дуги большого круга между двумя точками в градусах.
**Пример**
```sql
SELECT greatCircleAngle(0, 0, 45, 0) AS arc
```
```text
┌─arc─┐
│ 45 │
└─────┘
```
## pointInEllipses
Проверяет, принадлежит ли точка хотя бы одному из эллипсов.
@ -157,6 +189,121 @@ SELECT geohashDecode('ezs42') AS res
└─────────────────────────────────┘
```
## h3IsValid
Проверяет корректность H3-индекса.
```sql
h3IsValid(h3index)
```
**Входные значения**
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
**Возвращаемые значения**
- 0 — число не является H3-индексом
- 1 — число является H3-индексом
Тип — [UInt8](../../data_types/int_uint.md).
**Пример**
```sql
SELECT h3IsValid(630814730351855103) as h3IsValid
```
```text
┌─h3IsValid─┐
│ 1 │
└───────────┘
```
## h3GetResolution
Извлекает разрешение H3-индекса.
```sql
h3GetResolution(h3index)
```
**Входные значения**
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
**Возвращаемые значения**
- Разрешение сетки, от 0 до 15.
- Для несуществующего идентификатора может быть возвращено произвольное значение, используйте [h3IsValid](#h3IsValid) для проверки идентификаторов
Тип — [UInt8](../../data_types/int_uint.md).
**Пример**
```sql
SELECT h3GetResolution(639821929606596015) as resolution
```
```text
┌─resolution─┐
│ 14 │
└────────────┘
```
## h3EdgeAngle
Информирует о среднем размере стороны шестигранника H3 в градусах
```sql
h3EdgeAngle(resolution)
```
**Входные значения**
- `resolution` — требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значений — `[0, 15]`.
**Возвращаемые значения**
Средняя длина стороны многоугольника H3 в градусах, тип — [Float64](../../data_types/float.md).
**Пример**
```sql
SELECT h3EdgeAngle(10) as edgeAngle
```
```text
┌─────────h3EdgeAngle(10)─┐
│ 0.0005927224846720883 │
└───────────────────────┘
```
## h3EdgeLengthM
Информирует о среднем размере стороны шестигранника H3 в метрах
```sql
h3EdgeLengthM(resolution)
```
**Входные значения**
- `resolution` — требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значений — `[0, 15]`.
**Возвращаемые значения**
Средняя длина стороны многоугольника H3 в метрах, тип — [Float64](../../data_types/float.md).
**Пример**
```sql
SELECT h3EdgeLengthM(15) as edgeLengthM
```
```text
┌─edgeLengthM─┐
│ 0.509713273 │
└─────────────┘
```
## geoToH3
Получает H3 индекс точки `(lon, lat)` с заданным разрешением
@ -189,4 +336,38 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index
└────────────────────┘
```
## h3kRing
Возвращает H3-индексы шестиугольников в радиусе `k` от данного в произвольном порядке
```sql
h3kRing(h3index, k)
```
**Входные значения**
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
- `k` — радиус. Тип данных — [целое число](../../data_types/int_uint.md)
**Возвращаемые значения**
[Массив](../../data_types/array.md) из H3-индексов типа [UInt64](../../data_types/int_uint.md).
**Пример**
```sql
SELECT arrayJoin(h3kRing(644325529233966508, 1)) AS h3index
```
```text
┌────────────h3index─┐
│ 644325529233966508 │
│ 644325529233966497 │
│ 644325529233966510 │
│ 644325529233966504 │
│ 644325529233966509 │
│ 644325529233966355 │
│ 644325529233966354 │
└────────────────────┘
```
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/geo/) <!--hide-->