h3 remaining funcs checkAndGetColumn

This commit is contained in:
bharatnc 2022-01-22 22:58:21 -08:00
parent d2bc0712f7
commit 7d62593bb1
4 changed files with 61 additions and 12 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 UInt64.",
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 = getResolution(hindex);

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(