mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
h3 explicitly checkAndGetColumn
This commit is contained in:
parent
d1431fffd2
commit
d2bc0712f7
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user