Merge pull request #33921 from bharatnc/ncb/h3-improve

explicitly checkAndGetColumn for remaining H3 funcs
This commit is contained in:
Maksim Kita 2022-01-23 18:08:30 +01:00 committed by GitHub
commit 818966c1a2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 205 additions and 48 deletions

View File

@ -8,7 +8,6 @@
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
#include <base/range.h>
#include <constants.h>
#include <h3api.h>
@ -20,6 +19,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -52,7 +52,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt8>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt8.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto dst = ColumnVector<Float64>::create();
auto & dst_data = dst->getData();
@ -60,7 +69,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt8 resolution = col_hindex->getUInt(row);
const UInt8 resolution = data[row];
if (resolution > MAX_H3_RES)
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,

View File

@ -17,6 +17,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -49,7 +50,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
@ -57,7 +67,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex = col_hindex->getUInt(row);
const UInt64 hindex = data[row];
UInt8 res = getBaseCellNumber(hindex);

View File

@ -17,6 +17,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -49,7 +50,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
@ -57,7 +67,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex = col_hindex->getUInt(row);
const UInt64 hindex = data[row];
UInt8 res = getResolution(hindex);

View File

@ -17,6 +17,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -56,8 +57,27 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex_origin = arguments[0].column.get();
const auto * col_hindex_dest = arguments[1].column.get();
const auto * col_hindex_origin = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!col_hindex_origin)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_origin = col_hindex_origin->getData();
const auto * col_hindex_dest = checkAndGetColumn<ColumnUInt64>(arguments[1].column.get());
if (!col_hindex_dest)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[1].type->getName(),
2,
getName());
const auto & data_hindex_dest = col_hindex_dest->getData();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
@ -65,8 +85,8 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex_origin = col_hindex_origin->getUInt(row);
const UInt64 hindex_dest = col_hindex_dest->getUInt(row);
const UInt64 hindex_origin = data_hindex_origin[row];
const UInt64 hindex_dest = data_hindex_dest[row];
UInt8 res = areNeighborCells(hindex_origin, hindex_dest);

View File

@ -7,7 +7,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <base/range.h>
#include <h3api.h>
@ -17,6 +16,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -49,7 +49,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
@ -57,7 +66,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex = col_hindex->getUInt(row);
const UInt64 hindex = data[row];
UInt8 is_valid = isValidCell(hindex) == 0 ? 0 : 1;

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -65,8 +66,28 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * col_resolution = arguments[1].column.get();
const auto * col_hindex = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!col_hindex)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex = col_hindex->getData();
const auto * col_resolution = checkAndGetColumn<ColumnUInt8>(arguments[1].column.get());
if (!col_resolution)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt8.",
arguments[1].type->getName(),
2,
getName());
const auto & data_resolution = col_resolution->getData();
auto dst = ColumnArray::create(ColumnUInt64::create());
auto & dst_data = dst->getData();
@ -76,8 +97,8 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 parent_hindex = col_hindex->getUInt(row);
const UInt8 child_resolution = col_resolution->getUInt(row);
const UInt64 parent_hindex = data_hindex[row];
const UInt8 child_resolution = data_resolution[row];
if (child_resolution > MAX_H3_RES)
throw Exception(

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int INCORRECT_DATA;
extern const int ILLEGAL_COLUMN;
}
class FunctionH3ToGeoBoundary : public IFunction
@ -51,7 +52,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto latitude = ColumnFloat64::create();
auto longitude = ColumnFloat64::create();
@ -61,7 +71,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
H3Index h3index = col_hindex->getUInt(row);
H3Index h3index = data[row];
CellBoundary boundary{};
auto err = cellToBoundary(h3index, &boundary);

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -59,8 +60,27 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * col_resolution = arguments[1].column.get();
const auto * col_hindex = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!col_hindex)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex = col_hindex->getData();
const auto * col_resolution = checkAndGetColumn<ColumnUInt8>(arguments[1].column.get());
if (!col_resolution)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt8.",
arguments[1].type->getName(),
2,
getName());
const auto & data_resolution = col_resolution->getData();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
@ -68,14 +88,16 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex = col_hindex->getUInt(row);
const UInt8 resolution = col_resolution->getUInt(row);
const UInt64 hindex = data_hindex[row];
const UInt8 resolution = data_resolution[row];
if (resolution > MAX_H3_RES)
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));
toString(resolution),
getName(),
toString(MAX_H3_RES));
UInt64 res = cellToParent(hindex, resolution);

View File

@ -3,6 +3,7 @@
#if USE_H3
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
@ -17,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -51,7 +53,17 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto col_res = ColumnString::create();
auto & vec_res = col_res->getChars();
@ -63,9 +75,9 @@ public:
char * begin = reinterpret_cast<char *>(vec_res.data());
char * pos = begin;
for (size_t i = 0; i < input_rows_count; ++i)
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 hindex = col_hindex->getUInt(i);
const UInt64 hindex = data[row];
if (!isValidCell(hindex))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", hindex);
@ -76,7 +88,7 @@ public:
while (*pos != '\0')
pos++;
vec_offsets[i] = ++pos - begin;
vec_offsets[row] = ++pos - begin;
}
vec_res.resize(pos - begin);
return col_res;

View File

@ -11,7 +11,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <base/range.h>
#include <h3api.h>
@ -23,6 +22,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int PARAMETER_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -51,19 +51,41 @@ public:
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!isInteger(arg))
if (!WhichDataType(arg).isUInt16())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be integer",
arg->getName(), 2, getName());
"Illegal type {} of argument {} of function {}. Must be UInt16",
arg->getName(),
2,
getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_hindex = arguments[0].column.get();
const auto * col_k = arguments[1].column.get();
const auto * col_hindex = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!col_hindex)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex = col_hindex->getData();
/// ColumnUInt16 is sufficient as the max value of 2nd arg is checked (arg > 0 < 10000) in implementation below
const auto * col_k = checkAndGetColumn<ColumnUInt16>(arguments[1].column.get());
if (!col_k)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt16.",
arguments[1].type->getName(),
2,
getName());
const auto & data_k = col_k->getData();
auto dst = ColumnArray::create(ColumnUInt64::create());
auto & dst_data = dst->getData();
@ -73,8 +95,8 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
const H3Index origin_hindex = col_hindex->getUInt(row);
const int k = col_k->getInt(row);
const H3Index origin_hindex = data_hindex[row];
const int k = data_k[row];
/// Overflow is possible. The function maxGridDiskSize does not check for overflow.
/// The calculation is similar to square of k but several times more.
@ -82,6 +104,7 @@ public:
constexpr auto max_k = 10000;
if (k > max_k)
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Too large 'k' argument for {} function, maximum {}", getName(), max_k);
/// Check is already made while fetching the argument for k (to determine if it's an unsigned integer). Nevertheless, it's checked again here.
if (k < 0)
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Argument 'k' for {} function must be non negative", getName());

View File

@ -21,6 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -58,7 +59,16 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto * col_index = arguments[0].column.get();
const auto * column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
if (!column)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data = column->getData();
auto latitude = ColumnFloat64::create(input_rows_count);
auto longitude = ColumnFloat64::create(input_rows_count);
@ -69,7 +79,7 @@ public:
for (size_t row = 0; row < input_rows_count; ++row)
{
H3Index h3index = col_index->getUInt(row);
H3Index h3index = data[row];
LatLng coord{};
cellToLatLng(h3index,&coord);

View File

@ -1,16 +1,17 @@
-- Tags: no-fasttest
SELECT arraySort(h3kRing(581276613233082367, 1));
SELECT h3kRing(581276613233082367, 0);
SELECT h3kRing(581276613233082367, -1); -- { serverError 12 }
SELECT arraySort(h3kRing(581276613233082367, toUInt16(1)));
SELECT h3kRing(581276613233082367, toUInt16(0));
SELECT h3kRing(581276613233082367, -1); -- { serverError 43 }
SELECT h3kRing(581276613233082367, toUInt16(-1)); -- { serverError 12 }
DROP TABLE IF EXISTS h3_indexes;
CREATE TABLE h3_indexes (h3_index UInt64, res UInt8) ENGINE = Memory;
-- Test h3 indices and k selected from original test fixture: https://github.com/uber/h3/blob/master/src/apps/testapps
CREATE TABLE h3_indexes (h3_index UInt64, k UInt16) 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,1);
INSERT INTO h3_indexes VALUES (581263419093549055,2);
@ -30,6 +31,6 @@ INSERT INTO h3_indexes VALUES (639763125756281263,15);
INSERT INTO h3_indexes VALUES (644178757620501158,16);
SELECT arraySort(h3kRing(h3_index, res)) FROM h3_indexes ORDER BY h3_index;
SELECT arraySort(h3kRing(h3_index, k)) FROM h3_indexes ORDER BY h3_index;
DROP TABLE h3_indexes;

View File

@ -1,6 +1,6 @@
-- Tags: no-fasttest
SELECT h3kRing(581276613233082367, 65535); -- { serverError 12 }
SELECT h3kRing(581276613233082367, -1); -- { serverError 12 }
SELECT h3kRing(581276613233082367, -1); -- { serverError 43 }
SELECT length(h3kRing(111111111111, 1000));
SELECT h3kRing(581276613233082367, nan); -- { serverError 43 }

View File

@ -8,5 +8,5 @@ SELECT h3kRing(0xFFFFFFFFF, 1000) FORMAT Null;
SELECT h3kRing(0xFFFFFFFFFFFFFF, 1000) FORMAT Null;
SELECT h3GetBaseCell(0xFFFFFFFFFFFFFF) FORMAT Null;
SELECT h3GetResolution(0xFFFFFFFFFFFFFF) FORMAT Null;
SELECT h3kRing(0xFFFFFFFFFFFFFF, 10) FORMAT Null;
SELECT h3kRing(0xFFFFFFFFFFFFFF, toUInt16(10)) FORMAT Null;
SELECT h3ToGeo(0xFFFFFFFFFFFFFF) FORMAT Null;