mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Implementation of geohashEncode and geohashDecode functions;
Function signatures: * geohashEncode(float lon, float lat, int precision) => string * geohashDecode(string encoded) => (float llon, float lat) With test cases and documentation;
This commit is contained in:
parent
2abf420fac
commit
e4988110ec
@ -573,6 +573,13 @@ inline bool isInteger(const T & data_type)
|
||||
return which.isInt() || which.isUInt();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline bool isFloat(const T & data_type)
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
return which.isFloat();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline bool isNumber(const T & data_type)
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -13,6 +14,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column)
|
||||
@ -99,4 +101,21 @@ Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & ar
|
||||
return createBlockWithNestedColumnsImpl(block, args_set);
|
||||
}
|
||||
|
||||
void validateArgumentType(const IFunction & func, const DataTypes & arguments,
|
||||
size_t argument_index, bool (* validator_func)(const IDataType &),
|
||||
const char * expected_type_description)
|
||||
{
|
||||
if (arguments.size() <= argument_index)
|
||||
throw Exception("Incorrect number of arguments of function " + func.getName(),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto & argument = arguments[argument_index];
|
||||
if (validator_func(*argument) == false)
|
||||
throw Exception("Illegal type " + argument->getName() +
|
||||
" of " + std::to_string(argument_index) +
|
||||
" argument of function " + func.getName() +
|
||||
" expected " + expected_type_description,
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IFunction;
|
||||
|
||||
/// Methods, that helps dispatching over real column types.
|
||||
|
||||
template <typename Type>
|
||||
@ -64,7 +66,6 @@ bool checkColumnConst(const IColumn * column)
|
||||
return checkAndGetColumnConst<Type>(column);
|
||||
}
|
||||
|
||||
|
||||
/// Returns non-nullptr if column is ColumnConst with ColumnString or ColumnFixedString inside.
|
||||
const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column);
|
||||
|
||||
@ -94,4 +95,10 @@ Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & ar
|
||||
/// Similar function as above. Additionally transform the result type if needed.
|
||||
Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & args, size_t result);
|
||||
|
||||
/// Checks argument type at specified index with predicate.
|
||||
/// throws if there is no argument at specified index or if predicate returns false.
|
||||
void validateArgumentType(const IFunction & func, const DataTypes & arguments,
|
||||
size_t argument_index, bool (* validator_func)(const IDataType &),
|
||||
const char * expected_type_description);
|
||||
|
||||
}
|
||||
|
@ -1,19 +1,24 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsGeo.h>
|
||||
#include <Functions/GeoUtils.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <boost/geometry.hpp>
|
||||
#include <boost/geometry/geometries/point_xy.hpp>
|
||||
#include <boost/geometry/geometries/polygon.hpp>
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/ObjectPool.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
@ -246,6 +251,186 @@ private:
|
||||
|
||||
};
|
||||
|
||||
const size_t GEOHASH_MAX_TEXT_LENGTH = 16;
|
||||
|
||||
// geohashEncode(lon float32/64, lat float32/64, length UInt8) => string
|
||||
class FunctionGeohashEncode : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "geohashEncode";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashEncode>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
validateArgumentType(*this, arguments, 0, isFloat, "float");
|
||||
validateArgumentType(*this, arguments, 1, isFloat, "float");
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
validateArgumentType(*this, arguments, 2, isInteger, "integer");
|
||||
}
|
||||
if (arguments.size() > 3)
|
||||
{
|
||||
throw Exception("Too many arguments for function " + getName() +
|
||||
" expected at most 3",
|
||||
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
template <typename LonType, typename LatType>
|
||||
bool tryExecute(const IColumn * lon_column, const IColumn * lat_column, UInt64 precision_value, ColumnPtr & result)
|
||||
{
|
||||
const ColumnVector<LonType> * longitude = checkAndGetColumn<ColumnVector<LonType>>(lon_column);
|
||||
const ColumnVector<LatType> * latitude = checkAndGetColumn<ColumnVector<LatType>>(lat_column);
|
||||
if (!latitude || !longitude)
|
||||
return false;
|
||||
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const size_t size = lat_column->size();
|
||||
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * (GEOHASH_MAX_TEXT_LENGTH + 1));
|
||||
|
||||
char * begin = reinterpret_cast<char *>(out_vec.data());
|
||||
char * pos = begin;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const Float64 longitude_value = longitude->getElement(i);
|
||||
const Float64 latitude_value = latitude->getElement(i);
|
||||
|
||||
const size_t encoded_size = GeoUtils::geohashEncode(longitude_value, latitude_value, precision_value, pos);
|
||||
|
||||
pos += encoded_size;
|
||||
*pos = '\0';
|
||||
out_offsets[i] = ++pos - begin;
|
||||
}
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
result = std::move(col_str);
|
||||
|
||||
return true;
|
||||
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const IColumn * longitude = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * latitude = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
const UInt64 precision_value = std::min(GEOHASH_MAX_TEXT_LENGTH,
|
||||
arguments.size() == 3 ? block.getByPosition(arguments[2]).column->get64(0) : GEOHASH_MAX_TEXT_LENGTH);
|
||||
|
||||
ColumnPtr & res_column = block.getByPosition(result).column;
|
||||
|
||||
if (tryExecute<Float32, Float32>(longitude, latitude, precision_value, res_column) ||
|
||||
tryExecute<Float64, Float32>(longitude, latitude, precision_value, res_column) ||
|
||||
tryExecute<Float32, Float64>(longitude, latitude, precision_value, res_column) ||
|
||||
tryExecute<Float64, Float64>(longitude, latitude, precision_value, res_column))
|
||||
return;
|
||||
|
||||
const char sep[] = ", ";
|
||||
std::string arguments_description = "";
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
arguments_description += block.getByPosition(arguments[i]).column->getName() + sep;
|
||||
}
|
||||
if (arguments_description.size() > sizeof(sep))
|
||||
{
|
||||
arguments_description.erase(arguments_description.size() - sizeof(sep) - 1);
|
||||
}
|
||||
|
||||
throw Exception("Unsupported argument types: " + arguments_description +
|
||||
+ " for function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
// geohashDecode(string) => (lon float64, lat float64)
|
||||
class FunctionGeohashDecode : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "geohashDecode";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGeohashDecode>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
validateArgumentType(*this, arguments, 0, isStringOrFixedString, "string or fixed string");
|
||||
|
||||
return std::make_shared<DataTypeTuple>(
|
||||
DataTypes{std::make_shared<DataTypeFloat64>(), std::make_shared<DataTypeFloat64>()},
|
||||
Strings{"longitude", "latitude"});
|
||||
}
|
||||
|
||||
template <typename ColumnTypeEncoded>
|
||||
bool tryExecute(const IColumn * encoded_column, ColumnPtr & result_column)
|
||||
{
|
||||
const auto * encoded = checkAndGetColumn<ColumnTypeEncoded>(encoded_column);
|
||||
if (!encoded)
|
||||
return false;
|
||||
|
||||
const size_t count = encoded->size();
|
||||
|
||||
auto latitude = ColumnFloat64::create(count);
|
||||
auto longitude = ColumnFloat64::create(count);
|
||||
|
||||
ColumnFloat64::Container & lon_data = longitude->getData();
|
||||
ColumnFloat64::Container & lat_data = latitude->getData();
|
||||
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
{
|
||||
StringRef encoded_string = encoded->getDataAt(i);
|
||||
GeoUtils::geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]);
|
||||
}
|
||||
|
||||
MutableColumns result;
|
||||
result.emplace_back(std::move(longitude));
|
||||
result.emplace_back(std::move(latitude));
|
||||
result_column = ColumnTuple::create(std::move(result));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const IColumn * encoded = block.getByPosition(arguments[0]).column.get();
|
||||
ColumnPtr & res_column = block.getByPosition(result).column;
|
||||
|
||||
if (tryExecute<ColumnString>(encoded, res_column) ||
|
||||
tryExecute<ColumnFixedString>(encoded, res_column))
|
||||
return;
|
||||
|
||||
throw Exception("Unsupported argument type:" + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Type>
|
||||
using Point = boost::geometry::model::d2::point_xy<Type>;
|
||||
|
||||
@ -261,5 +446,7 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionPointInEllipses>();
|
||||
|
||||
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid, true>>();
|
||||
factory.registerFunction<FunctionGeohashEncode>();
|
||||
factory.registerFunction<FunctionGeohashDecode>();
|
||||
}
|
||||
}
|
||||
|
232
dbms/src/Functions/GeoUtils.cpp
Normal file
232
dbms/src/Functions/GeoUtils.cpp
Normal file
@ -0,0 +1,232 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Functions/GeoUtils.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
const char geohash_base32_encode_lookup_table[32] = {
|
||||
'0', '1', '2', '3', '4', '5', '6', '7', '8', '9',
|
||||
'b', 'c', 'd', 'e', 'f', 'g', 'h', 'j', 'k', 'm',
|
||||
'n', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x',
|
||||
'y', 'z',
|
||||
};
|
||||
|
||||
// TODO: this could be halved by excluding 128-255 range.
|
||||
const UInt8 geohash_base32_decode_lookup_table[256] = {
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 10, 11, 12, 13, 14, 15, 16, 0xFF, 17, 18, 0xFF, 19, 20, 0xFF,
|
||||
21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF,
|
||||
};
|
||||
|
||||
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;
|
||||
|
||||
using Encoded = std::array<UInt8, MAX_BITS>;
|
||||
|
||||
enum CoordType
|
||||
{
|
||||
LATITUDE,
|
||||
LONGITUDE,
|
||||
};
|
||||
|
||||
inline UInt8 singleCoordBitsPrecision(UInt8 precision, CoordType type)
|
||||
{
|
||||
// Single coordinate occupies only half of the total bits.
|
||||
const UInt8 bits = (precision * BITS_PER_SYMBOL) / 2;
|
||||
if (precision & 0x1 && type == LONGITUDE)
|
||||
{
|
||||
return bits + 1;
|
||||
}
|
||||
|
||||
return bits;
|
||||
}
|
||||
|
||||
inline Encoded encodeCoordinate(Float64 coord, Float64 min, Float64 max, UInt8 bits)
|
||||
{
|
||||
Encoded result;
|
||||
result.fill(0);
|
||||
|
||||
for (int i = 0; i < bits; ++i)
|
||||
{
|
||||
Float64 mid = (max + min) / 2;
|
||||
if (coord >= mid)
|
||||
{
|
||||
result[i] = 1;
|
||||
min = mid;
|
||||
}
|
||||
else
|
||||
{
|
||||
result[i] = 0;
|
||||
max = mid;
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
inline Float64 decodeCoordinate(const Encoded & coord, Float64 min, Float64 max, UInt8 bits)
|
||||
{
|
||||
Float64 mid = (max + min) / 2;
|
||||
for (int i = 0; i < bits; ++i)
|
||||
{
|
||||
const auto c = coord[i];
|
||||
if (c == 1)
|
||||
{
|
||||
min = mid;
|
||||
}
|
||||
else
|
||||
{
|
||||
max = mid;
|
||||
}
|
||||
|
||||
mid = (max + min) / 2;
|
||||
}
|
||||
|
||||
return mid;
|
||||
}
|
||||
|
||||
inline Encoded merge(const Encoded & encodedLon, const Encoded & encodedLat, UInt8 precision)
|
||||
{
|
||||
Encoded result;
|
||||
result.fill(0);
|
||||
|
||||
const auto bits = (precision * BITS_PER_SYMBOL) / 2;
|
||||
UInt8 i = 0;
|
||||
for (; i < bits; ++i)
|
||||
{
|
||||
result[i * 2 + 0] = encodedLon[i];
|
||||
result[i * 2 + 1] = encodedLat[i];
|
||||
}
|
||||
// in case of even precision, add last bit of longitude
|
||||
if (precision & 0x1)
|
||||
{
|
||||
result[i * 2] = encodedLon[i];
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
inline std::tuple<Encoded, Encoded> split(const Encoded & combined, UInt8 precision)
|
||||
{
|
||||
Encoded lat, lon;
|
||||
lat.fill(0);
|
||||
lon.fill(0);
|
||||
|
||||
UInt8 i = 0;
|
||||
for (; i < precision * BITS_PER_SYMBOL - 1; i += 2)
|
||||
{
|
||||
// longitude is even bits
|
||||
lon[i/2] = combined[i];
|
||||
lat[i/2] = combined[i + 1];
|
||||
}
|
||||
// precision is even, read the last bit as lat.
|
||||
if (precision & 0x1)
|
||||
{
|
||||
lon[i/2] = combined[precision * BITS_PER_SYMBOL - 1];
|
||||
}
|
||||
|
||||
return std::tie(lon, lat);
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
UInt8 v = binary[i];
|
||||
v <<= 1;
|
||||
v |= binary[i + 1];
|
||||
v <<= 1;
|
||||
v |= binary[i + 2];
|
||||
v <<= 1;
|
||||
v |= binary[i + 3];
|
||||
v <<= 1;
|
||||
v |= binary[i + 4];
|
||||
|
||||
assert(v < 32);
|
||||
|
||||
*out = geohash_base32_encode_lookup_table[v];
|
||||
++out;
|
||||
}
|
||||
}
|
||||
|
||||
inline Encoded base32Decode(const char * encoded_string, size_t encoded_length)
|
||||
{
|
||||
extern const UInt8 geohash_base32_decode_lookup_table[256];
|
||||
|
||||
Encoded result;
|
||||
|
||||
for (size_t i = 0; i < encoded_length; ++i)
|
||||
{
|
||||
const UInt8 c = static_cast<UInt8>(encoded_string[i]);
|
||||
const UInt8 decoded = geohash_base32_decode_lookup_table[c] & 0x1F;
|
||||
result[i * 5 + 4] = (decoded >> 0) & 0x01;
|
||||
result[i * 5 + 3] = (decoded >> 1) & 0x01;
|
||||
result[i * 5 + 2] = (decoded >> 2) & 0x01;
|
||||
result[i * 5 + 1] = (decoded >> 3) & 0x01;
|
||||
result[i * 5 + 0] = (decoded >> 4) & 0x01;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace GeoUtils
|
||||
{
|
||||
|
||||
size_t geohashEncode(Float64 longitude, Float64 latitude, UInt8 precision, char *& out)
|
||||
{
|
||||
if (precision == 0 || precision > MAX_PRECISION)
|
||||
{
|
||||
precision = MAX_PRECISION;
|
||||
}
|
||||
|
||||
const Encoded combined = merge(
|
||||
encodeCoordinate(longitude, -180, 180, singleCoordBitsPrecision(precision, LONGITUDE)),
|
||||
encodeCoordinate(latitude, -90, 90, singleCoordBitsPrecision(precision, LATITUDE)),
|
||||
precision);
|
||||
|
||||
base32Encode(combined, precision, out);
|
||||
|
||||
return precision;
|
||||
}
|
||||
|
||||
void geohashDecode(const char * encoded_string, size_t encoded_len, Float64 * longitude, Float64 * latitude)
|
||||
{
|
||||
const UInt8 precision = std::min(encoded_len, MAX_PRECISION);
|
||||
if (precision == 0)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
} // namespace GeoUtils
|
||||
|
||||
} // namespace DB
|
@ -699,6 +699,10 @@ std::string serialize(Polygon && polygon)
|
||||
return result;
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
|
||||
} /// GeoUtils
|
||||
|
||||
|
29
dbms/tests/performance/funtions_geo/functions_geo.xml
Normal file
29
dbms/tests/performance/funtions_geo/functions_geo.xml
Normal file
@ -0,0 +1,29 @@
|
||||
<test>
|
||||
<name>functions_coding</name>
|
||||
<type>once</type>
|
||||
|
||||
<stop_conditions>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>300</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>1000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<metrics>
|
||||
<max_rows_per_second />
|
||||
<max_bytes_per_second />
|
||||
<avg_rows_per_second />
|
||||
<avg_bytes_per_second />
|
||||
</metrics>
|
||||
|
||||
<!-- lat swings roughly from -90 to 90, lon swings from -180 to 180, and their tempo does not match. -->
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode((number % 150)*1.1 - 75, (number * 3.14 % 300)*1.1 - 150))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(toString(number % 1000000)))</query>
|
||||
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode(1.0/rand(), 2.0/rand()))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(toString(rand() % 1000000)))</query>
|
||||
|
||||
<!-- erroneus values -->
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode(number + 91.0, number + 181.0))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(hex(number)))</query>
|
||||
</test>
|
227
dbms/tests/queries/0_stateless/00932_geohash_support.reference
Normal file
227
dbms/tests/queries/0_stateless/00932_geohash_support.reference
Normal file
@ -0,0 +1,227 @@
|
||||
invalid values:
|
||||
zzzzzzzzzzzz
|
||||
000000000000
|
||||
1
|
||||
constant values:
|
||||
ezs42d000000
|
||||
-5.60303 42.60498
|
||||
default precision:
|
||||
ezs42d000000
|
||||
mixing const and non-const-columns:
|
||||
ezs42d000000
|
||||
from table (with const precision):
|
||||
1 6 Ok
|
||||
1 6 Ok
|
||||
1 6 Ok
|
||||
1 7 Ok
|
||||
1 7 Ok
|
||||
1 k Ok
|
||||
1 k Ok
|
||||
1 k Ok
|
||||
1 7 Ok
|
||||
1 7 Ok
|
||||
1 k Ok
|
||||
1 k Ok
|
||||
1 k Ok
|
||||
1 e Ok
|
||||
1 e Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 e Ok
|
||||
1 e Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 e Ok
|
||||
1 e Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 s Ok
|
||||
1 w Ok
|
||||
1 g Ok
|
||||
1 u Ok
|
||||
2 6g Ok
|
||||
2 6g Ok
|
||||
2 6g Ok
|
||||
2 7z Ok
|
||||
2 7z Ok
|
||||
2 kp Ok
|
||||
2 kp Ok
|
||||
2 kp Ok
|
||||
2 7z Ok
|
||||
2 7z Ok
|
||||
2 kp Ok
|
||||
2 kp Ok
|
||||
2 kp Ok
|
||||
2 eb Ok
|
||||
2 eb Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 eb Ok
|
||||
2 eb Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 eb Ok
|
||||
2 eb Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 s0 Ok
|
||||
2 w1 Ok
|
||||
2 gc Ok
|
||||
2 u9 Ok
|
||||
3 6gk Ok
|
||||
3 6gk Ok
|
||||
3 6gk Ok
|
||||
3 7zz Ok
|
||||
3 7zz Ok
|
||||
3 kpb Ok
|
||||
3 kpb Ok
|
||||
3 kpb Ok
|
||||
3 7zz Ok
|
||||
3 7zz Ok
|
||||
3 kpb Ok
|
||||
3 kpb Ok
|
||||
3 kpb Ok
|
||||
3 ebp Ok
|
||||
3 ebp Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 ebp Ok
|
||||
3 ebp Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 ebp Ok
|
||||
3 ebp Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 s00 Ok
|
||||
3 w1m Ok
|
||||
3 gcp Ok
|
||||
3 u9e Ok
|
||||
4 6gkz Ok
|
||||
4 6gkz Ok
|
||||
4 6gkz Ok
|
||||
4 7zzz Ok
|
||||
4 7zzz Ok
|
||||
4 kpbp Ok
|
||||
4 kpbp Ok
|
||||
4 kpbp Ok
|
||||
4 7zzz Ok
|
||||
4 7zzz Ok
|
||||
4 kpbp Ok
|
||||
4 kpbp Ok
|
||||
4 kpbp Ok
|
||||
4 ebpb Ok
|
||||
4 ebpb Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 ebpb Ok
|
||||
4 ebpb Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 ebpb Ok
|
||||
4 ebpb Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 s000 Ok
|
||||
4 w1mu Ok
|
||||
4 gcpv Ok
|
||||
4 u9ed Ok
|
||||
5 6gkzm Ok
|
||||
5 6gkzw Ok
|
||||
5 6gkzw Ok
|
||||
5 7zzzm Ok
|
||||
5 7zzzr Ok
|
||||
5 kpbp2 Ok
|
||||
5 kpbp2 Ok
|
||||
5 kpbp6 Ok
|
||||
5 7zzzv Ok
|
||||
5 7zzzz Ok
|
||||
5 kpbpb Ok
|
||||
5 kpbpb Ok
|
||||
5 kpbpf Ok
|
||||
5 ebpbj Ok
|
||||
5 ebpbp Ok
|
||||
5 s0000 Ok
|
||||
5 s0004 Ok
|
||||
5 ebpbj Ok
|
||||
5 ebpbp Ok
|
||||
5 s0000 Ok
|
||||
5 s0000 Ok
|
||||
5 s0004 Ok
|
||||
5 ebpbt Ok
|
||||
5 ebpbx Ok
|
||||
5 s0008 Ok
|
||||
5 s0008 Ok
|
||||
5 s000d Ok
|
||||
5 w1muy Ok
|
||||
5 gcpvn Ok
|
||||
5 u9edu Ok
|
||||
6 6gkzmg Ok
|
||||
6 6gkzwg Ok
|
||||
6 6gkzwg Ok
|
||||
6 7zzzrv Ok
|
||||
6 kpbp2j Ok
|
||||
6 7zzzvw Ok
|
||||
6 7zzzzy Ok
|
||||
6 kpbpbn Ok
|
||||
6 kpbpbn Ok
|
||||
6 kpbpfq Ok
|
||||
6 ebpbpb Ok
|
||||
6 s00000 Ok
|
||||
6 ebpbj9 Ok
|
||||
6 ebpbpc Ok
|
||||
6 s00001 Ok
|
||||
6 s00001 Ok
|
||||
6 s00043 Ok
|
||||
6 ebpbxf Ok
|
||||
6 s00084 Ok
|
||||
6 w1muy6 Ok
|
||||
6 gcpvn5 Ok
|
||||
6 u9edu0 Ok
|
||||
7 6gkzmg1 Ok
|
||||
7 6gkzwgj Ok
|
||||
7 6gkzwgj Ok
|
||||
7 7zzzrvb Ok
|
||||
7 kpbp2jz Ok
|
||||
7 7zzzzy0 Ok
|
||||
7 kpbpbnp Ok
|
||||
7 ebpbpb0 Ok
|
||||
7 s00000p Ok
|
||||
7 ebpbpcb Ok
|
||||
7 s00001z Ok
|
||||
7 ebpbxf0 Ok
|
||||
7 s00084p Ok
|
||||
7 w1muy6d Ok
|
||||
7 gcpvn5w Ok
|
||||
7 u9edu0q Ok
|
||||
8 6gkzmg1u Ok
|
||||
8 6gkzwgjt Ok
|
||||
8 6gkzwgjz Ok
|
||||
8 w1muy6dt Ok
|
||||
8 gcpvn5w2 Ok
|
||||
8 u9edu0qs Ok
|
||||
9 6gkzwgjzn Ok
|
||||
9 w1muy6dt2 Ok
|
||||
9 gcpvn5w2e Ok
|
||||
9 u9edu0qsf Ok
|
||||
10 6gkzwgjzn8 Ok
|
||||
10 w1muy6dt2p Ok
|
||||
10 gcpvn5w2eu Ok
|
||||
10 u9edu0qsf7 Ok
|
||||
11 6gkzwgjzn82 Ok
|
||||
11 w1muy6dt2pt Ok
|
||||
11 gcpvn5w2euk Ok
|
||||
11 u9edu0qsf7d Ok
|
||||
12 6gkzwgjzn820 Ok
|
||||
12 w1muy6dt2ptk Ok
|
||||
12 gcpvn5w2euky Ok
|
||||
12 u9edu0qsf7dn Ok
|
||||
incorrectly decoded values:
|
67
dbms/tests/queries/0_stateless/00932_geohash_support.sql
Normal file
67
dbms/tests/queries/0_stateless/00932_geohash_support.sql
Normal file
File diff suppressed because one or more lines are too long
@ -98,5 +98,57 @@ SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res
|
||||
└─────┘
|
||||
```
|
||||
|
||||
## geohashEncode
|
||||
|
||||
Encodes latitude and longitude as a geohash-string, please see (http://geohash.org/, https://en.wikipedia.org/wiki/Geohash).
|
||||
```
|
||||
geohashEncode(longitude, latitude, [precision])
|
||||
```
|
||||
|
||||
**Input values**
|
||||
|
||||
- longitude - longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`
|
||||
- latitude - latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`
|
||||
- precision - Optional, length of the resulting encoded string, defaults to `12`. Integer in range `[1, 12]`. Any value less than `1` or greater than `12` is silently converted to `12`.
|
||||
|
||||
**Returned values**
|
||||
|
||||
- alphanumeric `String` of encoded coordinate (modified version of the base32-encoding alphabet is used).
|
||||
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res
|
||||
```
|
||||
|
||||
```
|
||||
┌─res──────────┐
|
||||
│ ezs42d000000 │
|
||||
└──────────────┘
|
||||
```
|
||||
|
||||
## geohashDecode
|
||||
|
||||
Decodes any geohash-encoded string into longitude and latitude.
|
||||
|
||||
**Input values**
|
||||
|
||||
- encoded string - geohash-encoded string.
|
||||
|
||||
**Returned values**
|
||||
|
||||
- (longitude, latitude) - 2-tuple of `Float64` values of longitude and latitude.
|
||||
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
SELECT geohashDecode('ezs42') AS res
|
||||
```
|
||||
|
||||
```
|
||||
┌─res─────────────────────────────┐
|
||||
│ (-5.60302734375,42.60498046875) │
|
||||
└─────────────────────────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/geo/) <!--hide-->
|
||||
|
Loading…
Reference in New Issue
Block a user