From e4988110ec11500a83c618e0f2e772883a906179 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 9 Apr 2019 22:19:30 +0300 Subject: [PATCH] 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; --- dbms/src/DataTypes/IDataType.h | 7 + dbms/src/Functions/FunctionHelpers.cpp | 19 ++ dbms/src/Functions/FunctionHelpers.h | 9 +- dbms/src/Functions/FunctionsGeo.cpp | 199 ++++++++++++++- dbms/src/Functions/GeoUtils.cpp | 232 ++++++++++++++++++ dbms/src/Functions/GeoUtils.h | 4 + .../funtions_geo/functions_geo.xml | 29 +++ .../00932_geohash_support.reference | 227 +++++++++++++++++ .../0_stateless/00932_geohash_support.sql | 67 +++++ docs/en/query_language/functions/geo.md | 52 ++++ 10 files changed, 838 insertions(+), 7 deletions(-) create mode 100644 dbms/src/Functions/GeoUtils.cpp create mode 100644 dbms/tests/performance/funtions_geo/functions_geo.xml create mode 100644 dbms/tests/queries/0_stateless/00932_geohash_support.reference create mode 100644 dbms/tests/queries/0_stateless/00932_geohash_support.sql diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 60124cd3d5d..ad03b3cdd48 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -573,6 +573,13 @@ inline bool isInteger(const T & data_type) return which.isInt() || which.isUInt(); } +template +inline bool isFloat(const T & data_type) +{ + WhichDataType which(data_type); + return which.isFloat(); +} + template inline bool isNumber(const T & data_type) { diff --git a/dbms/src/Functions/FunctionHelpers.cpp b/dbms/src/Functions/FunctionHelpers.cpp index 1a05d3c2bfe..42fcebd5848 100644 --- a/dbms/src/Functions/FunctionHelpers.cpp +++ b/dbms/src/Functions/FunctionHelpers.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -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); +} + } diff --git a/dbms/src/Functions/FunctionHelpers.h b/dbms/src/Functions/FunctionHelpers.h index 6be58f32c5e..8e1efd113f8 100644 --- a/dbms/src/Functions/FunctionHelpers.h +++ b/dbms/src/Functions/FunctionHelpers.h @@ -12,6 +12,8 @@ namespace DB { +class IFunction; + /// Methods, that helps dispatching over real column types. template @@ -64,7 +66,6 @@ bool checkColumnConst(const IColumn * column) return checkAndGetColumnConst(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); + } diff --git a/dbms/src/Functions/FunctionsGeo.cpp b/dbms/src/Functions/FunctionsGeo.cpp index 0d13914f103..7b45fb8a668 100644 --- a/dbms/src/Functions/FunctionsGeo.cpp +++ b/dbms/src/Functions/FunctionsGeo.cpp @@ -1,19 +1,24 @@ #include #include #include +#include #include #include #include -#include -#include -#include -#include -#include #include -#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include + #include #include @@ -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(); } + + 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(); + } + + template + bool tryExecute(const IColumn * lon_column, const IColumn * lat_column, UInt64 precision_value, ColumnPtr & result) + { + const ColumnVector * longitude = checkAndGetColumn>(lon_column); + const ColumnVector * latitude = checkAndGetColumn>(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(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(longitude, latitude, precision_value, res_column) || + tryExecute(longitude, latitude, precision_value, res_column) || + tryExecute(longitude, latitude, precision_value, res_column) || + tryExecute(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(); } + + 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( + DataTypes{std::make_shared(), std::make_shared()}, + Strings{"longitude", "latitude"}); + } + + template + bool tryExecute(const IColumn * encoded_column, ColumnPtr & result_column) + { + const auto * encoded = checkAndGetColumn(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(encoded, res_column) || + tryExecute(encoded, res_column)) + return; + + throw Exception("Unsupported argument type:" + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + template using Point = boost::geometry::model::d2::point_xy; @@ -261,5 +446,7 @@ void registerFunctionsGeo(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction>(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/GeoUtils.cpp b/dbms/src/Functions/GeoUtils.cpp new file mode 100644 index 00000000000..8624b1c3b45 --- /dev/null +++ b/dbms/src/Functions/GeoUtils.cpp @@ -0,0 +1,232 @@ +#include +#include + +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; + +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 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(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 diff --git a/dbms/src/Functions/GeoUtils.h b/dbms/src/Functions/GeoUtils.h index 425377f17ae..731305cd705 100644 --- a/dbms/src/Functions/GeoUtils.h +++ b/dbms/src/Functions/GeoUtils.h @@ -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 diff --git a/dbms/tests/performance/funtions_geo/functions_geo.xml b/dbms/tests/performance/funtions_geo/functions_geo.xml new file mode 100644 index 00000000000..b1ea38be447 --- /dev/null +++ b/dbms/tests/performance/funtions_geo/functions_geo.xml @@ -0,0 +1,29 @@ + + functions_coding + once + + + + 300 + 1000 + + + + + + + + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode((number % 150)*1.1 - 75, (number * 3.14 % 300)*1.1 - 150)) + SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(toString(number % 1000000))) + + SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode(1.0/rand(), 2.0/rand())) + SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(toString(rand() % 1000000))) + + + SELECT count() FROM system.numbers WHERE NOT ignore(geohashEncode(number + 91.0, number + 181.0)) + SELECT count() FROM system.numbers WHERE NOT ignore(geohashDecode(hex(number))) + diff --git a/dbms/tests/queries/0_stateless/00932_geohash_support.reference b/dbms/tests/queries/0_stateless/00932_geohash_support.reference new file mode 100644 index 00000000000..ffc290681c7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00932_geohash_support.reference @@ -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: diff --git a/dbms/tests/queries/0_stateless/00932_geohash_support.sql b/dbms/tests/queries/0_stateless/00932_geohash_support.sql new file mode 100644 index 00000000000..a477332b532 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00932_geohash_support.sql @@ -0,0 +1,67 @@ +drop table if exists geohash_test_data; + +create table geohash_test_data ( + latitude Float64, + longitude Float64, + encoded String +) engine = MergeTree order by (latitude, longitude, encoded); + +-- data obtained from geohash.com +insert into geohash_test_data values (-25.427, -49.315, '6'), (-25.427, -49.315, '6g'), (-25.427, -49.315, '6gk'), (-25.427, -49.315, '6gkz'), (-25.427, -49.315, '6gkzm'), (-25.427, -49.315, '6gkzmg'), (-25.427, -49.315, '6gkzmg1'), (-25.427, -49.315, '6gkzmg1u'), (-25.383, -49.266, '6'), (-25.383, -49.266, '6g'), (-25.383, -49.266, '6gk'), (-25.383, -49.266, '6gkz'), (-25.383, -49.266, '6gkzw'), (-25.383, -49.266, '6gkzwg'), (-25.383, -49.266, '6gkzwgj'), (-25.383, -49.266, '6gkzwgjt'), (-25.382708, -49.265506, '6'), (-25.382708, -49.265506, '6g'), (-25.382708, -49.265506, '6gk'), (-25.382708, -49.265506, '6gkz'), (-25.382708, -49.265506, '6gkzw'), (-25.382708, -49.265506, '6gkzwg'), (-25.382708, -49.265506, '6gkzwgj'), (-25.382708, -49.265506, '6gkzwgjz'), (-25.382708, -49.265506, '6gkzwgjzn'), (-25.382708, -49.265506, '6gkzwgjzn8'), (-25.382708, -49.265506, '6gkzwgjzn82'), (-25.382708, -49.265506, '6gkzwgjzn820'), (-0.1, -0.1, '7'), (-0.1, -0.1, '7z'), (-0.1, -0.1, '7zz'), (-0.1, -0.1, '7zzz'), (-0.1, -0.1, '7zzzm'), (-0.1, -0.01, '7'), (-0.1, -0.01, '7z'), (-0.1, -0.01, '7zz'), (-0.1, -0.01, '7zzz'), (-0.1, -0.01, '7zzzr'), (-0.1, -0.01, '7zzzrv'), (-0.1, -0.01, '7zzzrvb'), (-0.1, 0, 'k'), (-0.1, 0, 'kp'), (-0.1, 0, 'kpb'), (-0.1, 0, 'kpbp'), (-0.1, 0, 'kpbp2'), (-0.1, 0.01, 'k'), (-0.1, 0.01, 'kp'), (-0.1, 0.01, 'kpb'), (-0.1, 0.01, 'kpbp'), (-0.1, 0.01, 'kpbp2'), (-0.1, 0.01, 'kpbp2j'), (-0.1, 0.01, 'kpbp2jz'), (-0.1, 0.1, 'k'), (-0.1, 0.1, 'kp'), (-0.1, 0.1, 'kpb'), (-0.1, 0.1, 'kpbp'), (-0.1, 0.1, 'kpbp6'), (-0.01, -0.1, '7'), (-0.01, -0.1, '7z'), (-0.01, -0.1, '7zz'), (-0.01, -0.1, '7zzz'), (-0.01, -0.1, '7zzzv'), (-0.01, -0.1, '7zzzvw'), (-0.01, -0.01, '7'), (-0.01, -0.01, '7z'), (-0.01, -0.01, '7zz'), (-0.01, -0.01, '7zzz'), (-0.01, -0.01, '7zzzz'), (-0.01, -0.01, '7zzzzy'), (-0.01, -0.01, '7zzzzy0'), (-0.01, 0, 'k'), (-0.01, 0, 'kp'), (-0.01, 0, 'kpb'), (-0.01, 0, 'kpbp'), (-0.01, 0, 'kpbpb'), (-0.01, 0, 'kpbpbn'), (-0.01, 0.01, 'k'), (-0.01, 0.01, 'kp'), (-0.01, 0.01, 'kpb'), (-0.01, 0.01, 'kpbp'), (-0.01, 0.01, 'kpbpb'), (-0.01, 0.01, 'kpbpbn'), (-0.01, 0.01, 'kpbpbnp'), (-0.01, 0.1, 'k'), (-0.01, 0.1, 'kp'), (-0.01, 0.1, 'kpb'), (-0.01, 0.1, 'kpbp'), (-0.01, 0.1, 'kpbpf'), (-0.01, 0.1, 'kpbpfq'), (0, -0.1, 'e'), (0, -0.1, 'eb'), (0, -0.1, 'ebp'), (0, -0.1, 'ebpb'), (0, -0.1, 'ebpbj'), (0, -0.01, 'e'), (0, -0.01, 'eb'), (0, -0.01, 'ebp'), (0, -0.01, 'ebpb'), (0, -0.01, 'ebpbp'), (0, -0.01, 'ebpbpb'), (0, -0.01, 'ebpbpb0'), (0, 0, 's'), (0, 0, 's0'), (0, 0, 's00'), (0, 0, 's000'), (0, 0.01, 's'), (0, 0.01, 's0'), (0, 0.01, 's00'), (0, 0.01, 's000'), (0, 0.01, 's0000'), (0, 0.01, 's00000'), (0, 0.01, 's00000p'), (0, 0.1, 's'), (0, 0.1, 's0'), (0, 0.1, 's00'), (0, 0.1, 's000'), (0, 0.1, 's0004'), (0.01, -0.1, 'e'), (0.01, -0.1, 'eb'), (0.01, -0.1, 'ebp'), (0.01, -0.1, 'ebpb'), (0.01, -0.1, 'ebpbj'), (0.01, -0.1, 'ebpbj9'), (0.01, -0.01, 'e'), (0.01, -0.01, 'eb'), (0.01, -0.01, 'ebp'), (0.01, -0.01, 'ebpb'), (0.01, -0.01, 'ebpbp'), (0.01, -0.01, 'ebpbpc'), (0.01, -0.01, 'ebpbpcb'), (0.01, 0, 's'), (0.01, 0, 's0'), (0.01, 0, 's00'), (0.01, 0, 's000'), (0.01, 0, 's0000'), (0.01, 0, 's00001'), (0.01, 0.01, 's'), (0.01, 0.01, 's0'), (0.01, 0.01, 's00'), (0.01, 0.01, 's000'), (0.01, 0.01, 's0000'), (0.01, 0.01, 's00001'), (0.01, 0.01, 's00001z'), (0.01, 0.1, 's'), (0.01, 0.1, 's0'), (0.01, 0.1, 's00'), (0.01, 0.1, 's000'), (0.01, 0.1, 's0004'), (0.01, 0.1, 's00043'), (0.1, -0.1, 'e'), (0.1, -0.1, 'eb'), (0.1, -0.1, 'ebp'), (0.1, -0.1, 'ebpb'), (0.1, -0.1, 'ebpbt'), (0.1, -0.01, 'e'), (0.1, -0.01, 'eb'), (0.1, -0.01, 'ebp'), (0.1, -0.01, 'ebpb'), (0.1, -0.01, 'ebpbx'), (0.1, -0.01, 'ebpbxf'), (0.1, -0.01, 'ebpbxf0'), (0.1, 0, 's'), (0.1, 0, 's0'), (0.1, 0, 's00'), (0.1, 0, 's000'), (0.1, 0, 's0008'), (0.1, 0.01, 's'), (0.1, 0.01, 's0'), (0.1, 0.01, 's00'), (0.1, 0.01, 's000'), (0.1, 0.01, 's0008'), (0.1, 0.01, 's00084'), (0.1, 0.01, 's00084p'), (0.1, 0.1, 's'), (0.1, 0.1, 's0'), (0.1, 0.1, 's00'), (0.1, 0.1, 's000'), (0.1, 0.1, 's000d'), (7.880886, 98.3640363, 'w'), (7.880886, 98.3640363, 'w1'), (7.880886, 98.3640363, 'w1m'), (7.880886, 98.3640363, 'w1mu'), (7.880886, 98.3640363, 'w1muy'), (7.880886, 98.3640363, 'w1muy6'), (7.880886, 98.3640363, 'w1muy6d'), (7.880886, 98.3640363, 'w1muy6dt'), (7.880886, 98.3640363, 'w1muy6dt2'), (7.880886, 98.3640363, 'w1muy6dt2p'), (7.880886, 98.3640363, 'w1muy6dt2pt'), (7.880886, 98.3640363, 'w1muy6dt2ptk'), (51.523242, -0.07914, 'g'), (51.523242, -0.07914, 'gc'), (51.523242, -0.07914, 'gcp'), (51.523242, -0.07914, 'gcpv'), (51.523242, -0.07914, 'gcpvn'), (51.523242, -0.07914, 'gcpvn5'), (51.523242, -0.07914, 'gcpvn5w'), (51.523242, -0.07914, 'gcpvn5w2'), (51.523242, -0.07914, 'gcpvn5w2e'), (51.523242, -0.07914, 'gcpvn5w2eu'), (51.523242, -0.07914, 'gcpvn5w2euk'), (51.523242, -0.07914, 'gcpvn5w2euky'), (53.923107, 27.606682, 'u'), (53.923107, 27.606682, 'u9'), (53.923107, 27.606682, 'u9e'), (53.923107, 27.606682, 'u9ed'), (53.923107, 27.606682, 'u9edu'), (53.923107, 27.606682, 'u9edu0'), (53.923107, 27.606682, 'u9edu0q'), (53.923107, 27.606682, 'u9edu0qs'), (53.923107, 27.606682, 'u9edu0qsf'), (53.923107, 27.606682, 'u9edu0qsf7'), (53.923107, 27.606682, 'u9edu0qsf7d'), (53.923107, 27.606682, 'u9edu0qsf7dn'); + + +select 'invalid values:'; -- must not crash +select geohashEncode(181.0, 91.0); +select geohashEncode(-181.0, -91.0); +select count(geohashDecode('abcdefghijklmnopqrstuvwxyz')); + +select 'constant values:'; +select geohashEncode(-5.60302734375, 42.593994140625, 0); +select round(geohashDecode('ezs42').1, 5), round(geohashDecode('ezs42').2, 5); + +select 'default precision:'; +select geohashEncode(-5.60302734375, 42.593994140625); + +select 'mixing const and non-const-columns:'; +select geohashEncode(materialize(-5.60302734375), materialize(42.593994140625), 0); +select geohashEncode(materialize(-5.60302734375), materialize(42.593994140625), materialize(0)); -- { serverError 44 } + + +select 'from table (with const precision):'; + +-- here results are strings, so reference may contain values to match for equality. +select 1 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 2 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 3 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 4 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 5 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 6 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 7 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 8 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 9 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 10 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 11 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; +select 12 as p, geohashEncode(longitude, latitude, p) as actual, if(actual = encoded, 'Ok', concat('expected: ', encoded)) from geohash_test_data WHERE length(encoded) = p; + +-- Here results are floats, and hence may not be compared for equality directly. +-- We select all values that are off by some reasonable value: +-- each byte of encoded string provides 5 bits of precison, (roughly 2.5 for lon and lat) +-- each bit of precision divides value range by 2. +-- hence max error is roughly value range 2.5 times divided by 2 for each precision bit. +-- initial value range is [-90..90] for latitude and [-180..180] for longitude. +select 'incorrectly decoded values:'; +select + geohashDecode(encoded) as actual, + 'expected:', encoded, '=>', latitude, longitude, + 'length:', length(encoded), + 'max lat error:', 180 / power(2, 2.5 * length(encoded)) as latitude_max_error, + 'max lon error:', 360 / power(2, 2.5 * length(encoded)) as longitude_max_error, + 'err:', (actual.2 - latitude) as lat_error, (actual.1 - longitude) as lon_error, + 'derr:', abs(lat_error) - latitude_max_error, abs(lon_error) - longitude_max_error +from geohash_test_data +where + abs(lat_error) > latitude_max_error + or + abs(lon_error) > longitude_max_error; + +drop table if exists geohash_test_data; diff --git a/docs/en/query_language/functions/geo.md b/docs/en/query_language/functions/geo.md index 2c9a3aac38f..97c4d1b5b4f 100644 --- a/docs/en/query_language/functions/geo.md +++ b/docs/en/query_language/functions/geo.md @@ -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/)