mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
check and get column remaining s2 funcs
This commit is contained in:
parent
8ba93354b0
commit
557cf6ecc1
@ -14,11 +14,12 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -62,9 +63,38 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * col_lo = arguments[0].column.get();
|
||||
const auto * col_hi = arguments[1].column.get();
|
||||
const auto * col_point = arguments[2].column.get();
|
||||
const auto * col_lo = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
|
||||
if (!col_lo)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[0].type->getName(),
|
||||
1,
|
||||
getName());
|
||||
|
||||
const auto & data_low = col_lo->getData();
|
||||
|
||||
const auto * col_hi = checkAndGetColumn<ColumnUInt64>(arguments[1].column.get());
|
||||
if (!col_hi)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[1].type->getName(),
|
||||
2,
|
||||
getName());
|
||||
|
||||
const auto & data_hi = col_hi->getData();
|
||||
|
||||
const auto * col_point = checkAndGetColumn<ColumnUInt64>(arguments[2].column.get());
|
||||
if (!col_point)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[2].type->getName(),
|
||||
3,
|
||||
getName());
|
||||
|
||||
const auto & data_point = col_point->getData();
|
||||
|
||||
auto dst = ColumnVector<UInt8>::create();
|
||||
auto & dst_data = dst->getData();
|
||||
@ -72,9 +102,9 @@ public:
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
const auto lo = S2CellId(col_lo->getUInt(row));
|
||||
const auto hi = S2CellId(col_hi->getUInt(row));
|
||||
const auto point = S2CellId(col_point->getUInt(row));
|
||||
const auto lo = S2CellId(data_low[row]);
|
||||
const auto hi = S2CellId(data_hi[row]);
|
||||
const auto point = S2CellId(data_point[row]);
|
||||
|
||||
if (!lo.is_valid() || !hi.is_valid())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Rectangle is not valid");
|
||||
|
@ -21,6 +21,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -67,10 +68,49 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * col_lo1 = arguments[0].column.get();
|
||||
const auto * col_hi1 = arguments[1].column.get();
|
||||
const auto * col_lo2 = arguments[2].column.get();
|
||||
const auto * col_hi2 = arguments[3].column.get();
|
||||
const auto * col_lo1 = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
|
||||
if (!col_lo1)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[0].type->getName(),
|
||||
1,
|
||||
getName());
|
||||
|
||||
const auto & data_lo1 = col_lo1->getData();
|
||||
|
||||
const auto * col_hi1 = checkAndGetColumn<ColumnFloat64>(arguments[1].column.get());
|
||||
if (!col_hi1)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[1].type->getName(),
|
||||
2,
|
||||
getName());
|
||||
|
||||
const auto & data_hi1 = col_hi1->getData();
|
||||
|
||||
const auto * col_lo2 = checkAndGetColumn<ColumnUInt64>(arguments[2].column.get());
|
||||
if (!col_lo2)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[2].type->getName(),
|
||||
3,
|
||||
getName());
|
||||
|
||||
const auto & data_lo2 = col_lo2->getData();
|
||||
|
||||
const auto * col_hi2 = checkAndGetColumn<ColumnFloat64>(arguments[3].column.get());
|
||||
if (!col_hi2)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[3].type->getName(),
|
||||
4,
|
||||
getName());
|
||||
|
||||
const auto & data_hi2 = col_hi2->getData();
|
||||
|
||||
auto col_res_first = ColumnUInt64::create();
|
||||
auto col_res_second = ColumnUInt64::create();
|
||||
@ -83,10 +123,10 @@ public:
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
const auto lo1 = S2CellId(col_lo1->getUInt(row));
|
||||
const auto hi1 = S2CellId(col_hi1->getUInt(row));
|
||||
const auto lo2 = S2CellId(col_lo2->getUInt(row));
|
||||
const auto hi2 = S2CellId(col_hi2->getUInt(row));
|
||||
const auto lo1 = S2CellId(data_lo1[row]);
|
||||
const auto hi1 = S2CellId(data_hi1[row]);
|
||||
const auto lo2 = S2CellId(data_lo2[row]);
|
||||
const auto hi2 = S2CellId(data_hi2[row]);
|
||||
|
||||
if (!lo1.is_valid() || !hi1.is_valid())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid");
|
||||
|
@ -19,6 +19,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -65,10 +66,49 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * col_lo1 = arguments[0].column.get();
|
||||
const auto * col_hi1 = arguments[1].column.get();
|
||||
const auto * col_lo2 = arguments[2].column.get();
|
||||
const auto * col_hi2 = arguments[3].column.get();
|
||||
const auto * col_lo1 = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
|
||||
if (!col_lo1)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[0].type->getName(),
|
||||
1,
|
||||
getName());
|
||||
|
||||
const auto & data_lo1 = col_lo1->getData();
|
||||
|
||||
const auto * col_hi1 = checkAndGetColumn<ColumnFloat64>(arguments[1].column.get());
|
||||
if (!col_hi1)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[1].type->getName(),
|
||||
2,
|
||||
getName());
|
||||
|
||||
const auto & data_hi1 = col_hi1->getData();
|
||||
|
||||
const auto * col_lo2 = checkAndGetColumn<ColumnUInt64>(arguments[2].column.get());
|
||||
if (!col_lo2)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[2].type->getName(),
|
||||
3,
|
||||
getName());
|
||||
|
||||
const auto & data_lo2 = col_lo2->getData();
|
||||
|
||||
const auto * col_hi2 = checkAndGetColumn<ColumnFloat64>(arguments[3].column.get());
|
||||
if (!col_hi2)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[3].type->getName(),
|
||||
4,
|
||||
getName());
|
||||
|
||||
const auto & data_hi2 = col_hi2->getData();
|
||||
|
||||
auto col_res_first = ColumnUInt64::create();
|
||||
auto col_res_second = ColumnUInt64::create();
|
||||
@ -81,10 +121,10 @@ public:
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
const auto lo1 = S2CellId(col_lo1->getUInt(row));
|
||||
const auto hi1 = S2CellId(col_hi1->getUInt(row));
|
||||
const auto lo2 = S2CellId(col_lo2->getUInt(row));
|
||||
const auto hi2 = S2CellId(col_hi2->getUInt(row));
|
||||
const auto lo1 = S2CellId(data_lo1[row]);
|
||||
const auto hi1 = S2CellId(data_hi1[row]);
|
||||
const auto lo2 = S2CellId(data_lo2[row]);
|
||||
const auto hi2 = S2CellId(data_hi2[row]);
|
||||
|
||||
if (!lo1.is_valid() || !hi1.is_valid())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid");
|
||||
|
@ -21,6 +21,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -67,7 +68,16 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto * col_id = arguments[0].column.get();
|
||||
const auto * col_id = checkAndGetColumn<ColumnFloat64>(arguments[0].column.get());
|
||||
if (!col_id)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal type {} of argument {} of function {}. Must be UInt64",
|
||||
arguments[0].type->getName(),
|
||||
1,
|
||||
getName());
|
||||
|
||||
const auto & data_id = col_id->getData();
|
||||
|
||||
auto col_longitude = ColumnFloat64::create();
|
||||
auto col_latitude = ColumnFloat64::create();
|
||||
@ -80,7 +90,7 @@ public:
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
const auto id = S2CellId(col_id->getUInt(row));
|
||||
const auto id = S2CellId(data_id[row]);
|
||||
|
||||
if (!id.is_valid())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid");
|
||||
|
Loading…
Reference in New Issue
Block a user