Merge pull request #58721 from ClickHouse/support-point-datatype-for-mysql

Support Point data type for MySQL
This commit is contained in:
Kseniia Sumarokova 2024-01-12 17:02:01 +01:00 committed by GitHub
commit 77f4cd8dd3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 106 additions and 2 deletions

View File

@ -10,7 +10,9 @@
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeCustomGeo.h>
#include <Common/typeid_cast.h>
#include <Common/logger_useful.h>
namespace DB
@ -41,6 +43,12 @@ void ExternalResultDescription::init(const Block & sample_block_)
DataTypePtr type_not_nullable = removeNullable(elem.type);
const IDataType * type = type_not_nullable.get();
if (dynamic_cast<const DataTypePointName *>(type->getCustomName()))
{
types.emplace_back(ValueType::vtPoint, is_nullable);
continue;
}
WhichDataType which(type);
if (which.isUInt8())

View File

@ -35,7 +35,8 @@ struct ExternalResultDescription
vtDecimal128,
vtDecimal256,
vtArray,
vtFixedString
vtFixedString,
vtPoint,
};
Block sample_block;

View File

@ -24,6 +24,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
}
@ -162,6 +163,8 @@ void insertPostgreSQLValue(
assert_cast<ColumnArray &>(column).insert(Array(dimensions[1].begin(), dimensions[1].end()));
break;
}
default:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported value type");
}
}

View File

@ -10,13 +10,15 @@
#include "DataTypeDate32.h"
#include "DataTypeDateTime.h"
#include "DataTypeDateTime64.h"
#include "DataTypeEnum.h"
#include "DataTypesDecimal.h"
#include "DataTypeFixedString.h"
#include "DataTypeNullable.h"
#include "DataTypeString.h"
#include "DataTypesNumber.h"
#include "DataTypeCustomGeo.h"
#include "DataTypeFactory.h"
#include "IDataType.h"
#include <Common/logger_useful.h>
namespace DB
{
@ -118,6 +120,10 @@ DataTypePtr convertMySQLDataType(MultiEnum<MySQLDataTypesSupport> type_support,
else if (precision <= DecimalUtils::max_precision<Decimal256>)
res = std::make_shared<DataTypeDecimal<Decimal256>>(precision, scale);
}
else if (type_name == "point")
{
res = DataTypeFactory::instance().get("Point");
}
/// Also String is fallback for all unknown types.
if (!res)

View File

@ -8,6 +8,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
@ -265,6 +266,42 @@ namespace
assert_cast<ColumnFixedString &>(column).insertData(value.data(), value.size());
read_bytes_size += column.sizeOfValueIfFixed();
break;
case ValueType::vtPoint:
{
/// The value is 25 bytes:
/// 4 bytes for integer SRID (0)
/// 1 byte for integer byte order (1 = little-endian)
/// 4 bytes for integer type information (1 = Point)
/// 8 bytes for double-precision X coordinate
/// 8 bytes for double-precision Y coordinate
ReadBufferFromMemory payload(value.data(), value.size());
String val;
payload.ignore(4);
UInt8 endian;
readBinary(endian, payload);
Int32 point_type;
readBinary(point_type, payload);
if (point_type != 1)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only Point data type is supported");
Float64 x, y;
if (endian == 1)
{
readBinaryLittleEndian(x, payload);
readBinaryLittleEndian(y, payload);
}
else
{
readBinaryBigEndian(x, payload);
readBinaryBigEndian(y, payload);
}
assert_cast<ColumnTuple &>(column).insert(Tuple({Field(x), Field(y)}));
read_bytes_size += value.size();
break;
}
default:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported value type");
}

View File

@ -859,6 +859,55 @@ def test_settings(started_cluster):
conn.close()
def test_mysql_point(started_cluster):
table_name = "test_mysql_point"
node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip)
drop_mysql_table(conn, table_name)
with conn.cursor() as cursor:
cursor.execute(
f"""
CREATE TABLE `clickhouse`.`{table_name}` (
`id` int NOT NULL,
`point` Point NOT NULL,
PRIMARY KEY (`id`)) ENGINE=InnoDB;
"""
)
cursor.execute(
f"INSERT INTO `clickhouse`.`{table_name}` SELECT 1, Point(15, 20)"
)
assert 1 == cursor.execute(f"SELECT count(*) FROM `clickhouse`.`{table_name}`")
conn.commit()
result = node1.query(
f"DESCRIBE mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
)
assert result.strip() == "id\tInt32\t\t\t\t\t\npoint\tPoint"
assert 1 == int(
node1.query(
f"SELECT count() FROM mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
)
)
assert (
"(15,20)"
== node1.query(
f"SELECT point FROM mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
).strip()
)
node1.query("DROP TABLE IF EXISTS test")
node1.query(
f"CREATE TABLE test (id Int32, point Point) Engine=MySQL('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
)
assert "(15,20)" == node1.query(f"SELECT point FROM test").strip()
drop_mysql_table(conn, table_name)
conn.close()
if __name__ == "__main__":
with contextmanager(started_cluster)() as cluster:
for name, instance in list(cluster.instances.items()):