mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Implement geohashesInBox function. #6127
This commit is contained in:
parent
f3dfd1c410
commit
fcb0482830
@ -36,6 +36,10 @@ const UInt8 geohash_base32_decode_lookup_table[256] = {
|
||||
const size_t BITS_PER_SYMBOL = 5;
|
||||
const size_t MAX_PRECISION = 12;
|
||||
const size_t MAX_BITS = MAX_PRECISION * BITS_PER_SYMBOL * 1.5;
|
||||
const Float64 LON_MIN = -180;
|
||||
const Float64 LON_MAX = 180;
|
||||
const Float64 LAT_MIN = -90;
|
||||
const Float64 LAT_MAX = 90;
|
||||
|
||||
using Encoded = std::array<UInt8, MAX_BITS>;
|
||||
|
||||
@ -64,7 +68,7 @@ inline Encoded encodeCoordinate(Float64 coord, Float64 min, Float64 max, UInt8 b
|
||||
|
||||
for (size_t i = 0; i < bits; ++i)
|
||||
{
|
||||
Float64 mid = (max + min) / 2;
|
||||
const Float64 mid = (max + min) / 2;
|
||||
if (coord >= mid)
|
||||
{
|
||||
result[i] = 1;
|
||||
@ -148,7 +152,7 @@ inline void base32Encode(const Encoded & binary, UInt8 precision, char * out)
|
||||
{
|
||||
extern const char geohash_base32_encode_lookup_table[32];
|
||||
|
||||
for (UInt8 i = 0; i < precision * BITS_PER_SYMBOL; i += 5)
|
||||
for (UInt8 i = 0; i < precision * BITS_PER_SYMBOL; i += BITS_PER_SYMBOL)
|
||||
{
|
||||
UInt8 v = binary[i];
|
||||
v <<= 1;
|
||||
@ -187,24 +191,38 @@ inline Encoded base32Decode(const char * encoded_string, size_t encoded_length)
|
||||
return result;
|
||||
}
|
||||
|
||||
inline Float64 getMaxSpan(CoordType type)
|
||||
{
|
||||
if (type == LONGITUDE)
|
||||
{
|
||||
return LON_MAX - LON_MIN;
|
||||
}
|
||||
|
||||
return LAT_MAX - LAT_MIN;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
inline Float64 getSpan(UInt8 precision, CoordType type)
|
||||
{
|
||||
const auto bits = singleCoordBitsPrecision(precision, type);
|
||||
// since every bit of precision divides span by 2, divide max span by 2^bits.
|
||||
return ldexp(getMaxSpan(type), -1 * bits);
|
||||
}
|
||||
|
||||
namespace GeoUtils
|
||||
{
|
||||
|
||||
size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char *& out)
|
||||
inline UInt8 geohashPrecision(UInt8 precision)
|
||||
{
|
||||
if (precision == 0 || precision > MAX_PRECISION)
|
||||
{
|
||||
precision = MAX_PRECISION;
|
||||
}
|
||||
|
||||
return precision;
|
||||
}
|
||||
|
||||
inline size_t geohashEncodeImpl(Float64 longitude, Float64 latitude, UInt8 precision, char * out)
|
||||
{
|
||||
const Encoded combined = merge(
|
||||
encodeCoordinate(longitude, -180, 180, singleCoordBitsPrecision(precision, LONGITUDE)),
|
||||
encodeCoordinate(latitude, -90, 90, singleCoordBitsPrecision(precision, LATITUDE)),
|
||||
encodeCoordinate(longitude, LON_MIN, LON_MAX, singleCoordBitsPrecision(precision, LONGITUDE)),
|
||||
encodeCoordinate(latitude, LAT_MIN, LAT_MAX, singleCoordBitsPrecision(precision, LATITUDE)),
|
||||
precision);
|
||||
|
||||
base32Encode(combined, precision, out);
|
||||
@ -212,9 +230,28 @@ size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char
|
||||
return precision;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
namespace GeoUtils
|
||||
{
|
||||
|
||||
size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char * out)
|
||||
{
|
||||
precision = geohashPrecision(precision);
|
||||
return geohashEncodeImpl(longitude, latitude, precision, out);
|
||||
}
|
||||
|
||||
void geohashDecode(const char * encoded_string, size_t encoded_len, Float64 * longitude, Float64 * latitude)
|
||||
{
|
||||
const UInt8 precision = std::min(encoded_len, MAX_PRECISION);
|
||||
const UInt8 precision = std::min(encoded_len, static_cast<size_t>(MAX_PRECISION));
|
||||
if (precision == 0)
|
||||
{
|
||||
return;
|
||||
@ -223,8 +260,89 @@ void geohashDecode(const char * encoded_string, size_t encoded_len, Float64 * lo
|
||||
Encoded lat_encoded, lon_encoded;
|
||||
std::tie(lon_encoded, lat_encoded) = split(base32Decode(encoded_string, precision), precision);
|
||||
|
||||
*longitude = decodeCoordinate(lon_encoded, -180, 180, singleCoordBitsPrecision(precision, LONGITUDE));
|
||||
*latitude = decodeCoordinate(lat_encoded, -90, 90, singleCoordBitsPrecision(precision, LATITUDE));
|
||||
*longitude = decodeCoordinate(lon_encoded, LON_MIN, LON_MAX, singleCoordBitsPrecision(precision, LONGITUDE));
|
||||
*latitude = decodeCoordinate(lat_encoded, LAT_MIN, LAT_MAX, singleCoordBitsPrecision(precision, LATITUDE));
|
||||
}
|
||||
|
||||
GeohashesInBoxPreparedArgs geohashesInBoxPrepare(const Float64 longitude_min,
|
||||
const Float64 latitude_min,
|
||||
const Float64 longitude_max,
|
||||
const Float64 latitude_max,
|
||||
UInt8 precision)
|
||||
{
|
||||
precision = geohashPrecision(precision);
|
||||
|
||||
if (longitude_max < longitude_min || latitude_max < latitude_min)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
const auto lon_step = getSpan(precision, LONGITUDE);
|
||||
const auto lat_step = getSpan(precision, LATITUDE);
|
||||
|
||||
// align max to the right(or up) border of geohash grid cell to ensure that cell is in result.
|
||||
Float64 lon_min = floor(longitude_min / lon_step) * lon_step;
|
||||
Float64 lat_min = floor(latitude_min / lat_step) * lat_step;
|
||||
Float64 lon_max = ceil(longitude_max / lon_step) * lon_step;
|
||||
Float64 lat_max = ceil(latitude_max / lat_step) * lat_step;
|
||||
|
||||
const auto lon_span = lon_max - lon_min;
|
||||
const auto lat_span = lat_max - lat_min;
|
||||
// in case of a very small (or zero) span, produce at least 1 item.
|
||||
const auto items_count = std::max(size_t{1}, static_cast<size_t>(ceil(lon_span/lon_step * lat_span/lat_step)));
|
||||
|
||||
return GeohashesInBoxPreparedArgs{
|
||||
items_count,
|
||||
precision,
|
||||
lon_min,
|
||||
lat_min,
|
||||
lon_max,
|
||||
lat_max,
|
||||
lon_step,
|
||||
lat_step
|
||||
};
|
||||
}
|
||||
|
||||
UInt64 geohashesInBox(const GeohashesInBoxPreparedArgs & args, char * out)
|
||||
{
|
||||
if (args.items_count == 0
|
||||
|| args.precision == 0
|
||||
|| args.precision > MAX_PRECISION
|
||||
|| args.latitude_min > args.latitude_max
|
||||
|| args.longitude_min > args.longitude_max
|
||||
|| args.longitude_step <= 0
|
||||
|| args.latitude_step <= 0)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
UInt64 items = 0;
|
||||
for (auto lon = args.longitude_min; lon < args.longitude_max; lon += args.longitude_step)
|
||||
{
|
||||
for (auto lat = args.latitude_min; lat < args.latitude_max; lat += args.latitude_step)
|
||||
{
|
||||
assert(items <= args.items_count);
|
||||
|
||||
size_t l = geohashEncodeImpl(lon, lat, args.precision, out);
|
||||
out += l;
|
||||
*out = '\0';
|
||||
++out;
|
||||
|
||||
++items;
|
||||
}
|
||||
}
|
||||
|
||||
if (items == 0 && args.items_count != 0)
|
||||
{
|
||||
size_t l = geohashEncodeImpl(args.longitude_min, args.latitude_min, args.precision, out);
|
||||
out += l;
|
||||
*out = '\0';
|
||||
++out;
|
||||
|
||||
++items;
|
||||
}
|
||||
|
||||
return items;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -706,10 +706,33 @@ std::string serialize(Polygon && polygon)
|
||||
return result;
|
||||
}
|
||||
|
||||
size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char *& out);
|
||||
size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char * out);
|
||||
|
||||
void geohashDecode(const char * encoded_string, size_t encoded_len, Float64 * longitude, Float64 * latitude);
|
||||
|
||||
std::vector<std::pair<Float64, Float64>> geohashCoverBox(Float64 longitude_min, Float64 latitude_min, Float64 longitude_max, Float64 latitude_max, UInt8 precision, UInt32 max_items = 0);
|
||||
|
||||
struct GeohashesInBoxPreparedArgs
|
||||
{
|
||||
UInt64 items_count = 0;
|
||||
UInt8 precision = 0;
|
||||
|
||||
Float64 longitude_min = 0.0;
|
||||
Float64 latitude_min = 0.0;
|
||||
Float64 longitude_max = 0.0;
|
||||
Float64 latitude_max = 0.0;
|
||||
|
||||
Float64 longitude_step = 0.0;
|
||||
Float64 latitude_step = 0.0;
|
||||
};
|
||||
|
||||
GeohashesInBoxPreparedArgs geohashesInBoxPrepare(const Float64 longitude_min,
|
||||
const Float64 latitude_min,
|
||||
Float64 longitude_max,
|
||||
Float64 latitude_max,
|
||||
UInt8 precision);
|
||||
|
||||
UInt64 geohashesInBox(const GeohashesInBoxPreparedArgs & estimation, char * out);
|
||||
|
||||
} /// GeoUtils
|
||||
|
||||
|
169
dbms/src/Functions/geohashesInBox.cpp
Normal file
169
dbms/src/Functions/geohashesInBox.cpp
Normal file
@ -0,0 +1,169 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/GeoUtils.h>
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||
}
|
||||
|
||||
class FunctionGeohashesInBox : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "geohashesInBox";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashesInBox>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 5; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
validateArgumentType(*this, arguments, 0, isFloat, "float");
|
||||
validateArgumentType(*this, arguments, 1, isFloat, "float");
|
||||
validateArgumentType(*this, arguments, 2, isFloat, "float");
|
||||
validateArgumentType(*this, arguments, 3, isFloat, "float");
|
||||
validateArgumentType(*this, arguments, 4, isUInt8, "integer");
|
||||
|
||||
if (!(arguments[0]->equals(*arguments[1]) &&
|
||||
arguments[0]->equals(*arguments[2]) &&
|
||||
arguments[0]->equals(*arguments[3])))
|
||||
{
|
||||
throw Exception("Illegal type of argument of " + getName() +
|
||||
" all coordinate arguments must have the same type, instead they are:" +
|
||||
arguments[0]->getName() + ", " +
|
||||
arguments[1]->getName() + ", " +
|
||||
arguments[2]->getName() + ", " +
|
||||
arguments[3]->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
template <typename LonAndLatType, typename PrecisionType>
|
||||
void execute(const IColumn * lon_min_column,
|
||||
const IColumn * lat_min_column,
|
||||
const IColumn * lon_max_column,
|
||||
const IColumn * lat_max_column,
|
||||
const IColumn * precision_column,
|
||||
ColumnPtr & result)
|
||||
{
|
||||
static constexpr size_t max_array_size = 10'000'000;
|
||||
|
||||
const auto * lon_min = checkAndGetColumn<ColumnVector<LonAndLatType>>(lon_min_column);
|
||||
const auto * lat_min = checkAndGetColumn<ColumnVector<LonAndLatType>>(lat_min_column);
|
||||
const auto * lon_max = checkAndGetColumn<ColumnVector<LonAndLatType>>(lon_max_column);
|
||||
const auto * lat_max = checkAndGetColumn<ColumnVector<LonAndLatType>>(lat_max_column);
|
||||
auto * precision = checkAndGetColumn<ColumnVector<PrecisionType>>(precision_column);
|
||||
if (precision == nullptr)
|
||||
{
|
||||
precision = checkAndGetColumnConstData<ColumnVector<PrecisionType>>(precision_column);
|
||||
}
|
||||
|
||||
if (!lon_min || !lat_min || !lon_max || !lat_max || !precision)
|
||||
{
|
||||
throw Exception("Unsupported argument types for function " + getName() + " : " +
|
||||
lon_min_column->getName() + ", " +
|
||||
lat_min_column->getName() + ", " +
|
||||
lon_max_column->getName() + ", " +
|
||||
lat_max_column->getName() + ".",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
const size_t total_rows = lat_min->size();
|
||||
|
||||
auto col_res = ColumnArray::create(ColumnString::create());
|
||||
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
|
||||
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
|
||||
ColumnString::Chars & res_strings_chars = res_strings.getChars();
|
||||
ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets();
|
||||
|
||||
for (size_t row = 0; row < total_rows; ++row)
|
||||
{
|
||||
const Float64 lon_min_value = lon_min->getElement(row);
|
||||
const Float64 lat_min_value = lat_min->getElement(row);
|
||||
const Float64 lon_max_value = lon_max->getElement(row);
|
||||
const Float64 lat_max_value = lat_max->getElement(row);
|
||||
|
||||
const auto prepared_args = GeoUtils::geohashesInBoxPrepare(
|
||||
lon_min_value, lat_min_value, lon_max_value, lat_max_value,
|
||||
precision->getElement(row % precision->size()));
|
||||
if (prepared_args.items_count > max_array_size)
|
||||
{
|
||||
throw Exception(getName() + " would produce " + std::to_string(prepared_args.items_count) +
|
||||
" array elements, which is bigger than the allowed maximum of " + std::to_string(max_array_size),
|
||||
ErrorCodes::TOO_LARGE_ARRAY_SIZE);
|
||||
}
|
||||
|
||||
res_strings_offsets.reserve(res_strings_offsets.size() + prepared_args.items_count);
|
||||
res_strings_chars.resize(res_strings_chars.size() + prepared_args.items_count * (prepared_args.precision + 1));
|
||||
const auto starting_offset = res_strings_offsets.empty() ? 0 : res_strings_offsets.back();
|
||||
char * out = reinterpret_cast<char *>(res_strings_chars.data() + starting_offset);
|
||||
|
||||
// Actually write geohashes into preallocated buffer.
|
||||
GeoUtils::geohashesInBox(prepared_args, out);
|
||||
|
||||
for (UInt8 i = 1; i <= prepared_args.items_count ; ++i)
|
||||
{
|
||||
res_strings_offsets.push_back(starting_offset + (prepared_args.precision + 1) * i);
|
||||
}
|
||||
res_offsets.push_back((res_offsets.empty() ? 0 : res_offsets.back()) + prepared_args.items_count);
|
||||
}
|
||||
if (!res_strings_offsets.empty() && res_strings_offsets.back() != res_strings_chars.size())
|
||||
{
|
||||
throw Exception("String column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (!res_offsets.empty() && res_offsets.back() != res_strings.size())
|
||||
{
|
||||
throw Exception("Arrary column size mismatch (internal logical error)" +
|
||||
std::to_string(res_offsets.back()) + " != " + std::to_string(res_strings.size()),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
result = std::move(col_res);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const IColumn * lon_min = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * lat_min = block.getByPosition(arguments[1]).column.get();
|
||||
const IColumn * lon_max = block.getByPosition(arguments[2]).column.get();
|
||||
const IColumn * lat_max = block.getByPosition(arguments[3]).column.get();
|
||||
const IColumn * prec = block.getByPosition(arguments[4]).column.get();
|
||||
ColumnPtr & res = block.getByPosition(result).column;
|
||||
|
||||
if (checkColumn<ColumnVector<Float32>>(lon_min))
|
||||
{
|
||||
execute<Float32, UInt8>(lon_min, lat_min, lon_max, lat_max, prec, res);
|
||||
}
|
||||
else
|
||||
{
|
||||
execute<Float64, UInt8>(lon_min, lat_min, lon_max, lat_max, prec, res);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionGeohashesInBox(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionGeohashesInBox>();
|
||||
}
|
||||
|
||||
}
|
@ -10,6 +10,7 @@ void registerFunctionPointInEllipses(FunctionFactory & factory);
|
||||
void registerFunctionPointInPolygon(FunctionFactory & factory);
|
||||
void registerFunctionGeohashEncode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashDecode(FunctionFactory & factory);
|
||||
void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
||||
|
||||
#if USE_H3
|
||||
void registerFunctionGeoToH3(FunctionFactory &);
|
||||
@ -22,6 +23,7 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
||||
registerFunctionPointInPolygon(factory);
|
||||
registerFunctionGeohashEncode(factory);
|
||||
registerFunctionGeohashDecode(factory);
|
||||
registerFunctionGeohashesInBox(factory);
|
||||
|
||||
#if USE_H3
|
||||
registerFunctionGeoToH3(factory);
|
||||
|
@ -0,0 +1,40 @@
|
||||
center
|
||||
['7zz','ebp','kpb','s00']
|
||||
['7zzzm','7zzzq','7zzzr','7zzzt','7zzzv','7zzzw','7zzzx','7zzzy','7zzzz','ebpbj','ebpbm','ebpbn','ebpbp','ebpbq','ebpbr','ebpbt','ebpbw','ebpbx','kpbp2','kpbp3','kpbp6','kpbp8','kpbp9','kpbpb','kpbpc','kpbpd','kpbpf','s0000','s0001','s0002','s0003','s0004','s0006','s0008','s0009','s000d']
|
||||
['7zzzz','ebpbp','kpbpb','s0000']
|
||||
north pole
|
||||
['bpb']
|
||||
['gzz']
|
||||
['upb']
|
||||
['zzz']
|
||||
south pole
|
||||
['000']
|
||||
['5bp']
|
||||
['h00']
|
||||
['pbp']
|
||||
wrap point around equator
|
||||
['rzz']
|
||||
['xbp']
|
||||
['2pb']
|
||||
['800']
|
||||
arbitrary values in all 4 quarters
|
||||
['w1muy4','w1muy5','w1muy6','w1muy7','w1muyh','w1muyk']
|
||||
['thym0','thym1','thym2','thym3','thym4','thym6','thym8','thym9','thymd']
|
||||
['6gkzx5','6gkzx7','6gkzxh','6gkzxj','6gkzxk','6gkzxm']
|
||||
['h927mu','h927mv','h927my','h927qh','h927qj','h927qn']
|
||||
small range always produces array of length 1
|
||||
zooming
|
||||
['s7']
|
||||
['s7w']
|
||||
['s7w1','s7w3','s7w4','s7w5','s7w6','s7w7','s7w9','s7wc','s7wd','s7we','s7wf','s7wg','s7wh','s7wj','s7wk','s7wm','s7wn','s7wp','s7wq','s7wr','s7ws','s7wt','s7wu','s7wv','s7ww','s7wx','s7wy','s7wz']
|
||||
['s7w1z','s7w3b','s7w3c','s7w3f','s7w3g','s7w3u','s7w4p','s7w4r','s7w4x','s7w4z','s7w5p','s7w60','s7w61','s7w62','s7w63','s7w64','s7w65','s7w66','s7w67','s7w68','s7w69','s7w6b','s7w6c','s7w6d','s7w6e','s7w6f','s7w6g','s7w6h','s7w6k','s7w6s','s7w6u','s7w70','s7w71','s7w74','s7w75','s7w7h']
|
||||
['s7w1z0','s7w1z1','s7w1z2','s7w1z3','s7w1z4','s7w1z5','s7w1z6','s7w1z7','s7w1z8','s7w1z9','s7w1zb','s7w1zc','s7w1zd','s7w1ze','s7w1zf','s7w1zg','s7w1zh','s7w1zj','s7w1zk','s7w1zm','s7w1zn','s7w1zp','s7w1zq','s7w1zr','s7w1zs','s7w1zt','s7w1zu','s7w1zv','s7w1zw','s7w1zx','s7w1zy','s7w1zz','s7w3b0','s7w3b1','s7w3b2','s7w3b3','s7w3b4','s7w3b5','s7w3b6','s7w3b7','s7w3b8','s7w3b9','s7w3bd','s7w3be','s7w3bh','s7w3bj','s7w3bk','s7w3bm','s7w3bn','s7w3bp','s7w3bq','s7w3br','s7w3bs','s7w3bt','s7w3bw','s7w3bx','s7w4p0','s7w4p1','s7w4p2','s7w4p3','s7w4p4','s7w4p5','s7w4p6','s7w4p7','s7w4p8','s7w4p9','s7w4pb','s7w4pc','s7w4pd','s7w4pe','s7w4pf','s7w4pg','s7w4ph','s7w4pk','s7w4ps','s7w4pu','s7w600','s7w601','s7w602','s7w603','s7w604','s7w605','s7w606','s7w607','s7w608','s7w609','s7w60d','s7w60e','s7w60h','s7w60k','s7w60s']
|
||||
['s7w1z0g','s7w1z0u','s7w1z0v','s7w1z0y','s7w1z0z','s7w1z15','s7w1z17','s7w1z1e','s7w1z1g','s7w1z1h','s7w1z1j','s7w1z1k','s7w1z1m','s7w1z1n','s7w1z1p','s7w1z1q','s7w1z1r','s7w1z1s','s7w1z1t','s7w1z1u','s7w1z1v','s7w1z1w','s7w1z1x','s7w1z1y','s7w1z1z','s7w1z2b','s7w1z2c','s7w1z2f','s7w1z30','s7w1z31','s7w1z32','s7w1z33','s7w1z34','s7w1z36','s7w1z38','s7w1z39','s7w1z3b','s7w1z3c','s7w1z3d','s7w1z3f','s7w1z45','s7w1z47','s7w1z4e','s7w1z4h','s7w1z4j','s7w1z4k','s7w1z4m','s7w1z4n','s7w1z4p','s7w1z4q','s7w1z4r','s7w1z4s','s7w1z4t','s7w1z4w','s7w1z4x','s7w1z60','s7w1z61','s7w1z62','s7w1z63','s7w1z64','s7w1z66','s7w1z68','s7w1z69','s7w1z6d']
|
||||
['s7w1z0gs','s7w1z0gt','s7w1z0gu','s7w1z0gv','s7w1z0gw','s7w1z0gx','s7w1z0gy','s7w1z0gz','s7w1z0uh','s7w1z0uj','s7w1z0uk','s7w1z0um','s7w1z0un','s7w1z0up','s7w1z0uq','s7w1z0ur','s7w1z158','s7w1z159','s7w1z15b','s7w1z15c','s7w1z15d','s7w1z15f','s7w1z1h0','s7w1z1h1','s7w1z1h2','s7w1z1h3','s7w1z1h4','s7w1z1h6']
|
||||
['s7w1z0gs3','s7w1z0gs6','s7w1z0gs7','s7w1z0gs9','s7w1z0gsc','s7w1z0gsd','s7w1z0gse','s7w1z0gsf','s7w1z0gsg','s7w1z0gsk','s7w1z0gss','s7w1z0gsu','s7w1z0gt1','s7w1z0gt4','s7w1z0gt5','s7w1z0gth']
|
||||
['s7w1z0gs3y','s7w1z0gs3z','s7w1z0gs6n','s7w1z0gs6p','s7w1z0gs9b','s7w1z0gsd0']
|
||||
['s7w1z0gs3y0','s7w1z0gs3y1','s7w1z0gs3y2','s7w1z0gs3y3']
|
||||
['s7w1z0gs3y0z','s7w1z0gs3y1p','s7w1z0gs3y1r','s7w1z0gs3y1x','s7w1z0gs3y2b','s7w1z0gs3y2c','s7w1z0gs3y2f','s7w1z0gs3y2g','s7w1z0gs3y2u','s7w1z0gs3y2v','s7w1z0gs3y30','s7w1z0gs3y31','s7w1z0gs3y32','s7w1z0gs3y33','s7w1z0gs3y34','s7w1z0gs3y35','s7w1z0gs3y36','s7w1z0gs3y37','s7w1z0gs3y38','s7w1z0gs3y39','s7w1z0gs3y3d','s7w1z0gs3y3e','s7w1z0gs3y3h','s7w1z0gs3y3j','s7w1z0gs3y3k','s7w1z0gs3y3m','s7w1z0gs3y3s','s7w1z0gs3y3t']
|
||||
['s7w1z0gs3y0z','s7w1z0gs3y1p','s7w1z0gs3y1r','s7w1z0gs3y1x','s7w1z0gs3y2b','s7w1z0gs3y2c','s7w1z0gs3y2f','s7w1z0gs3y2g','s7w1z0gs3y2u','s7w1z0gs3y2v','s7w1z0gs3y30','s7w1z0gs3y31','s7w1z0gs3y32','s7w1z0gs3y33','s7w1z0gs3y34','s7w1z0gs3y35','s7w1z0gs3y36','s7w1z0gs3y37','s7w1z0gs3y38','s7w1z0gs3y39','s7w1z0gs3y3d','s7w1z0gs3y3e','s7w1z0gs3y3h','s7w1z0gs3y3j','s7w1z0gs3y3k','s7w1z0gs3y3m','s7w1z0gs3y3s','s7w1z0gs3y3t']
|
||||
['s7w1z0gs3y0z','s7w1z0gs3y1p','s7w1z0gs3y1r','s7w1z0gs3y1x','s7w1z0gs3y2b','s7w1z0gs3y2c','s7w1z0gs3y2f','s7w1z0gs3y2g','s7w1z0gs3y2u','s7w1z0gs3y2v','s7w1z0gs3y30','s7w1z0gs3y31','s7w1z0gs3y32','s7w1z0gs3y33','s7w1z0gs3y34','s7w1z0gs3y35','s7w1z0gs3y36','s7w1z0gs3y37','s7w1z0gs3y38','s7w1z0gs3y39','s7w1z0gs3y3d','s7w1z0gs3y3e','s7w1z0gs3y3h','s7w1z0gs3y3j','s7w1z0gs3y3k','s7w1z0gs3y3m','s7w1z0gs3y3s','s7w1z0gs3y3t']
|
||||
errors
|
63
dbms/tests/queries/0_stateless/00972_geohashesInBox.sql
Normal file
63
dbms/tests/queries/0_stateless/00972_geohashesInBox.sql
Normal file
@ -0,0 +1,63 @@
|
||||
-- test data acquired with: https://github.com/sunng87/node-geohash
|
||||
-- geohash.bboxes(minlat, minlon, maxlat, maxlon, precision)
|
||||
-- as
|
||||
-- geohashesInBox(minlon, minlat, maxlon, maxlat, precision)
|
||||
-- except for the cases when JS-version produces result outside of given region,
|
||||
-- typically at wrap points: poles, 0-latitude and 0-longitude.
|
||||
|
||||
select 'center';
|
||||
SELECT arraySort(geohashesInBox(-1.0, -1.0, 1.0, 1.0, 3));
|
||||
SELECT arraySort(geohashesInBox(-0.1, -0.1, 0.1, 0.1, 5));
|
||||
SELECT arraySort(geohashesInBox(-0.01, -0.01, 0.01, 0.01, 5));
|
||||
|
||||
select 'north pole';
|
||||
SELECT arraySort(geohashesInBox(-180.0, 89.0, -179.0, 90.0, 3));
|
||||
SELECT arraySort(geohashesInBox(-1.0, 89.0, 0.0, 90.0, 3));
|
||||
SELECT arraySort(geohashesInBox(0.0, 89.0, 1.0, 90.0, 3));
|
||||
SELECT arraySort(geohashesInBox(179.0, 89.0, 180.0, 90.0, 3));
|
||||
|
||||
select 'south pole';
|
||||
SELECT arraySort(geohashesInBox(-180.0, -90.0, -179.0, -89.0, 3));
|
||||
SELECT arraySort(geohashesInBox(-1.0, -90.0, 0.0, -89.0, 3));
|
||||
SELECT arraySort(geohashesInBox(0.0, -90.0, 1.0, -89.0, 3));
|
||||
SELECT arraySort(geohashesInBox(179.0, -90.0, 180.0, -89.0, 3));
|
||||
|
||||
select 'wrap point around equator';
|
||||
SELECT arraySort(geohashesInBox(179.0, -1.0, 180.0, 0.0, 3));
|
||||
SELECT arraySort(geohashesInBox(179.0, 0.0, 180.0, 1.0, 3));
|
||||
SELECT arraySort(geohashesInBox(-180.0, -1.0, -179.0, 0.0, 3));
|
||||
SELECT arraySort(geohashesInBox(-180.0, 0.0, -179.0, 1.0, 3));
|
||||
|
||||
select 'arbitrary values in all 4 quarters';
|
||||
SELECT arraySort(geohashesInBox(98.36, 7.88, 98.37, 7.89, 6));
|
||||
SELECT arraySort(geohashesInBox(53.8, 27.6, 53.9, 27.7, 5));
|
||||
SELECT arraySort(geohashesInBox(-49.26, -25.38, -49.25, -25.37, 6));
|
||||
SELECT arraySort(geohashesInBox(23.11, -82.37, 23.12, -82.36, 6));
|
||||
|
||||
select 'small range always produces array of length 1';
|
||||
SELECT lon/5 - 180 as lon1, lat/5 - 90 as lat1, lon1 as lon2, lat1 as lat2, geohashesInBox(lon1, lat1, lon2, lat2, 1) as g FROM (SELECT arrayJoin(range(360*5)) as lon, arrayJoin(range(180*5)) as lat) WHERE length(g) != 1;
|
||||
SELECT lon/5 - 40 as lon1, lat/5 - 20 as lat1, lon1 as lon2, lat1 as lat2, geohashesInBox(lon1, lat1, lon2, lat2, 12) as g FROM (SELECT arrayJoin(range(80*5)) as lon, arrayJoin(range(10*5)) as lat) WHERE length(g) != 1;
|
||||
SELECT lon/5 - 40 as lon1, lat/5 - 20 as lat1, lon1 + 0.0000000001 as lon2, lat1 + 0.0000000001 as lat2, geohashesInBox(lon1, lat1, lon2, lat2, 1) as g FROM (SELECT arrayJoin(range(80*5)) as lon, arrayJoin(range(10*5)) as lat) WHERE length(g) != 1;
|
||||
|
||||
select 'zooming';
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 21.0, 21.0, 2));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 21.0, 21.0, 3));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 21.0, 21.0, 4));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.25, 20.25, 5));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.0625, 20.0625, 6));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.01, 20.01, 7));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.001, 20.001, 8));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.0001, 20.0001, 9));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.00001, 20.00001, 10));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.000001, 20.000001, 11));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.000001, 20.000001, 12));
|
||||
|
||||
-- precision greater than 12 is truncated to 12, so these two calls would produce same result as above
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.000001, 20.000001, 13));
|
||||
SELECT arraySort(geohashesInBox(20.0, 20.0, 20.000001, 20.000001, 14));
|
||||
|
||||
select 'errors';
|
||||
SELECT geohashesInBox(); -- { serverError 42 } -- not enough arguments
|
||||
SELECT geohashesInBox(1, 2, 3, 4, 5); -- { serverError 43 } -- wrong types of arguments
|
||||
SELECT geohashesInBox(toFloat32(1.0), 2.0, 3.0, 4.0, 5); -- { serverError 43 } -- all lats and longs should be of the same type
|
||||
SELECT geohashesInBox(24.48, 40.56, 24.785, 40.81, 12); -- { serverError 128 } -- to many elements in array
|
@ -183,4 +183,36 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index
|
||||
└────────────────────┘
|
||||
```
|
||||
|
||||
## geohashesInBox
|
||||
|
||||
Returns an array of geohash-encoded strings of given precision that fall inside and intersect boundaries of given box, basically a 2D grid flattened into array.
|
||||
|
||||
**Input values**
|
||||
|
||||
- longitude_min - min longitude, floating value in range `[-180°, 180°]`
|
||||
- latitude_min - min latitude, floating value in range `[-90°, 90°]`
|
||||
- longitude_max - max longitude, floating value in range `[-180°, 180°]`
|
||||
- latitude_max - max latitude, floating value in range `[-90°, 90°]`
|
||||
- precision - geohash precision, `UInt8` in range `[1, 12]`
|
||||
|
||||
Please note that all coordinate parameters should be of the same type: either `Float32` or `Float64`.
|
||||
|
||||
**Returned values**
|
||||
|
||||
- array of precision-long strings of geohash-boxes covering provided area, you should not rely on order of items.
|
||||
- [] - empty array if *min* values of *latitude* and *longitude* aren't less than corresponding *max* values.
|
||||
|
||||
Please note that function will throw an exception if resulting array is over 10'000'000 items long.
|
||||
|
||||
**Example**
|
||||
|
||||
```
|
||||
SELECT geohashesInBox(24.48, 40.56, 24.785, 40.81, 4) AS thasos
|
||||
```
|
||||
```
|
||||
┌─thasos──────────────────────────────────────┐
|
||||
│ ['sx1q','sx1r','sx32','sx1w','sx1x','sx38'] │
|
||||
└─────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/geo/) <!--hide-->
|
||||
|
Loading…
Reference in New Issue
Block a user