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:
Vasily Nemkov 2019-04-09 22:19:30 +03:00
parent 2abf420fac
commit e4988110ec
10 changed files with 838 additions and 7 deletions

View File

@ -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)
{

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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>();
}
}

View 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

View File

@ -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

View 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>

View 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:

File diff suppressed because one or more lines are too long

View File

@ -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-->