h3 explicitly checkAndGetColumn

This commit is contained in:
bharatnc 2022-01-22 22:25:06 -08:00
parent d1431fffd2
commit d2bc0712f7
7 changed files with 130 additions and 25 deletions

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
@ -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[0].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

@ -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[0].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);

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
@ -54,16 +54,37 @@ public:
if (!isInteger(arg))
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 Integer",
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();
const auto * col_k = checkAndGetColumn<ColumnInt64>(arguments[1].column.get());
if (!col_k)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be Integer.",
arguments[0].type->getName(),
2,
getName());
const auto & data_k = col_k->getData();
auto dst = ColumnArray::create(ColumnUInt64::create());
auto & dst_data = dst->getData();
@ -73,8 +94,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.

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);