Merge pull request #55479 from slvrtrn/fix-mysql-datetime-incompatibilities

Fix MySQL text protocol DateTime formatting and LowCardinality(Nullable(T)) types reporting
This commit is contained in:
robot-ch-test-poll1 2023-10-11 16:56:26 +02:00 committed by GitHub
commit 80aa9cad51
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 369 additions and 260 deletions

View File

@ -10,6 +10,5 @@ RUN curl -L -o /mysql-connector-j-${ver}.jar https://repo1.maven.org/maven2/com/
ENV CLASSPATH=$CLASSPATH:/mysql-connector-j-${ver}.jar
WORKDIR /jdbc
COPY Test.java Test.java
COPY PreparedStatementsTest.java PreparedStatementsTest.java
RUN javac Test.java PreparedStatementsTest.java
COPY MySQLJavaClientTest.java MySQLJavaClientTest.java
RUN javac MySQLJavaClientTest.java

View File

@ -2,7 +2,7 @@ import com.mysql.cj.MysqlType;
import java.sql.*;
public class PreparedStatementsTest {
public class MySQLJavaClientTest {
public static void main(String[] args) {
int i = 0;
String host = "127.0.0.1";
@ -10,6 +10,7 @@ public class PreparedStatementsTest {
String user = "default";
String password = "";
String database = "default";
String binary = "false";
while (i < args.length) {
switch (args[i]) {
case "--host":
@ -27,16 +28,19 @@ public class PreparedStatementsTest {
case "--database":
database = args[++i];
break;
case "--binary":
binary = args[++i];
break;
default:
i++;
break;
}
}
// useServerPrepStmts uses COM_STMT_PREPARE and COM_STMT_EXECUTE
// instead of COM_QUERY which allows us to test the binary protocol
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true",
host, port, database);
// useServerPrepStmts=true -> COM_STMT_PREPARE + COM_STMT_EXECUTE -> binary
// useServerPrepStmts=false -> COM_QUERY -> text
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=%s",
host, port, database, binary);
try {
Class.forName("com.mysql.cj.jdbc.Driver");
@ -49,6 +53,7 @@ public class PreparedStatementsTest {
testDateTypes(conn);
testUnusualDateTime64Scales(conn);
testDateTimeTimezones(conn);
testSuspiciousNullableLowCardinalityTypes(conn);
conn.close();
} catch (Exception e) {
e.printStackTrace();
@ -58,7 +63,7 @@ public class PreparedStatementsTest {
private static void testSimpleDataTypes(Connection conn) throws SQLException {
System.out.println("### testSimpleDataTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_simple_data_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM simple_data_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -83,7 +88,7 @@ public class PreparedStatementsTest {
private static void testStringTypes(Connection conn) throws SQLException {
System.out.println("### testStringTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_string_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM string_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -97,7 +102,7 @@ public class PreparedStatementsTest {
private static void testLowCardinalityAndNullableTypes(Connection conn) throws SQLException {
System.out.println("### testLowCardinalityAndNullableTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_low_cardinality_and_nullable_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM low_cardinality_and_nullable_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -111,7 +116,7 @@ public class PreparedStatementsTest {
private static void testDecimalTypes(Connection conn) throws SQLException {
System.out.println("### testDecimalTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM decimal_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -127,7 +132,7 @@ public class PreparedStatementsTest {
private static void testDateTypes(Connection conn) throws SQLException {
System.out.println("### testDateTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_date_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM date_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -143,7 +148,7 @@ public class PreparedStatementsTest {
private static void testUnusualDateTime64Scales(Connection conn) throws SQLException {
System.out.println("### testUnusualDateTime64Scales");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_unusual_datetime64_scales").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM unusual_datetime64_scales").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -160,7 +165,7 @@ public class PreparedStatementsTest {
private static void testDateTimeTimezones(Connection conn) throws SQLException {
System.out.println("### testDateTimeTimezones");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_datetime_timezones").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM datetime_timezones").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -172,7 +177,7 @@ public class PreparedStatementsTest {
private static void testMiscTypes(Connection conn) throws SQLException {
System.out.println("### testMiscTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_misc_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM misc_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -184,6 +189,20 @@ public class PreparedStatementsTest {
System.out.println();
}
private static void testSuspiciousNullableLowCardinalityTypes(Connection conn) throws SQLException {
System.out.println("### testSuspiciousNullableLowCardinalityTypes");
String query = "SELECT * FROM suspicious_nullable_low_cardinality_types";
ResultSet rs = conn.prepareStatement(query).executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
System.out.printf("%s, value: %s\n", getMysqlType(rs, "f"), rs.getFloat("f"));
System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d"));
System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"));
}
System.out.println();
}
private static String getMysqlType(ResultSet rs, String columnLabel) throws SQLException {
ResultSetMetaData meta = rs.getMetaData();
return String.format("%s type is %s", columnLabel,

View File

@ -1,78 +0,0 @@
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
class JavaConnectorTest {
private static final String CREATE_TABLE_SQL = "CREATE TABLE IF NOT EXISTS default.test1 (`age` Int32, `name` String, `int_nullable` Nullable(Int32)) Engine = Memory";
private static final String INSERT_SQL = "INSERT INTO default.test1(`age`, `name`) VALUES(33, 'jdbc'),(44, 'ck')";
private static final String SELECT_SQL = "SELECT * FROM default.test1";
private static final String SELECT_NUMBER_SQL = "SELECT * FROM system.numbers LIMIT 13";
private static final String DROP_TABLE_SQL = "DROP TABLE default.test1";
public static void main(String[] args) {
int i = 0;
String host = "127.0.0.1";
String port = "9004";
String user = "default";
String password = "";
String database = "default";
while (i < args.length) {
switch (args[i]) {
case "--host":
host = args[++i];
break;
case "--port":
port = args[++i];
break;
case "--user":
user = args[++i];
break;
case "--password":
password = args[++i];
break;
case "--database":
database = args[++i];
break;
default:
i++;
break;
}
}
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false", host, port, database);
Connection conn = null;
Statement stmt = null;
try {
Class.forName("com.mysql.cj.jdbc.Driver");
conn = DriverManager.getConnection(jdbcUrl, user, password);
stmt = conn.createStatement();
stmt.executeUpdate(CREATE_TABLE_SQL);
stmt.executeUpdate(INSERT_SQL);
ResultSet rs = stmt.executeQuery(SELECT_SQL);
while (rs.next()) {
System.out.print(rs.getString("age"));
System.out.print(rs.getString("name"));
System.out.print(rs.getString("int_nullable"));
System.out.println();
}
stmt.executeUpdate(DROP_TABLE_SQL);
rs = stmt.executeQuery(SELECT_NUMBER_SQL);
while (rs.next()) {
System.out.print(rs.getString(1));
System.out.println();
}
stmt.close();
conn.close();
} catch (Exception e) {
e.printStackTrace();
System.exit(1);
}
}
}

View File

@ -0,0 +1,65 @@
#include "Common/assert_cast.h"
#include "Columns/ColumnNullable.h"
#include "Columns/ColumnsDateTime.h"
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "base/types.h"
namespace DB
{
namespace MySQLProtocol
{
namespace MySQLUtils
{
DecimalUtils::DecimalComponents<DateTime64>
getNormalizedDateTime64Components(const DataTypePtr data_type, const ColumnPtr col, const int row_num)
{
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
auto components = DecimalUtils::split(value, scale);
using T = typename DateTime64::NativeType;
if (value.value < 0 && components.fractional)
{
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
--components.whole;
}
if (components.fractional != 0)
{
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
else
{
// fractional == 1 is a different microsecond value depending on the scale
// Scale 1 = 100000
// Scale 2 = 010000
// Scale 3 = 001000
// Scale 4 = 000100
// Scale 5 = 000010
// Scale 6 = 000001
components.fractional *= static_cast<int>(pow(10, 6 - scale));
}
}
return components;
};
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i)
{
ColumnPtr col = columns[i]->convertToFullIfNeeded();
if (col->isNullable())
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
return col;
};
}
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "base/types.h"
namespace DB
{
namespace MySQLProtocol
{
namespace MySQLUtils
{
/// Splits DateTime64 column data at a certain row number into whole and fractional part
/// Additionally, normalizes the fractional part as if it was scale 6 for MySQL compatibility purposes
DecimalUtils::DecimalComponents<DateTime64> getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, int row_num);
/// If a column is ColumnSparse/ColumnLowCardinality/ColumnNullable, it is unwrapped in a correct order;
/// otherwise, the original column is returned
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i);
}
}
}

View File

@ -15,6 +15,7 @@
#include "DataTypes/DataTypesNumber.h"
#include "Formats/FormatSettings.h"
#include "IO/WriteBufferFromString.h"
#include "MySQLUtils.h"
#include "base/DayNum.h"
#include "base/Decimal.h"
#include "base/types.h"
@ -32,7 +33,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
FormatSettings format_settings;
for (size_t i = 0; i < columns.size(); ++i)
{
ColumnPtr col = getColumn(i);
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
if (col->isNullAt(row_num))
{
// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row
@ -42,7 +43,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
continue; // NULLs are stored in the null bitmap only
}
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
TypeIndex type_index = data_type->getTypeId();
switch (type_index)
{
@ -95,13 +96,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
break;
}
case TypeIndex::DateTime64: {
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type)));
bool has_microseconds = components.fractional != 0;
@ -150,11 +145,11 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
buffer.write(null_bitmap.data(), null_bitmap_size);
for (size_t i = 0; i < columns.size(); ++i)
{
ColumnPtr col = getColumn(i);
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
if (col->isNullAt(row_num))
continue;
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
TypeIndex type_index = data_type->getTypeId();
switch (type_index)
{
@ -257,27 +252,7 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
break;
}
case TypeIndex::DateTime64: {
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
if (components.fractional != 0)
{
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
else
{
// fractional == 1 is a different microsecond value depending on the scale
// Scale 1 = 100000
// Scale 2 = 010000
// Scale 3 = 001000
// Scale 4 = 000100
// Scale 5 = 000010
// Scale 6 = 000001
components.fractional *= static_cast<int>(pow(10, 6 - scale));
}
}
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
String timezone = getDateTimeTimezone(*data_type);
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone));
UInt16 year = ldt.year();
@ -327,34 +302,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
}
}
}
ResultSetRow::DateTime64ComponentsWithScale ResultSetRow::getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const
{
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
auto components = DecimalUtils::split(value, scale);
using T = typename DateTime64::NativeType;
if (value.value < 0 && components.fractional)
{
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
--components.whole;
}
return {components, scale};
}
ColumnPtr ResultSetRow::getColumn(size_t i) const
{
ColumnPtr col = columns[i]->convertToFullIfNeeded();
if (col->isNullable())
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
return col;
}
}
}
}

View File

@ -16,12 +16,6 @@ namespace ProtocolBinary
{
class ResultSetRow : public IMySQLWritePacket
{
using DateTime64ComponentsWithScale = std::pair<DecimalUtils::DecimalComponents<DateTime64>, UInt32>;
private:
DateTime64ComponentsWithScale getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const;
ColumnPtr getColumn(size_t i) const;
protected:
int row_num;
const Columns & columns;

View File

@ -7,6 +7,7 @@
#include "DataTypes/DataTypeLowCardinality.h"
#include "DataTypes/DataTypeNullable.h"
#include "DataTypes/DataTypesDecimal.h"
#include "MySQLUtils.h"
namespace DB
{
@ -17,17 +18,32 @@ namespace MySQLProtocol
namespace ProtocolText
{
ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_)
ResultSetRow::ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_)
: columns(columns_), row_num(row_num_)
{
static FormatSettings format_settings = {.bool_true_representation = "1", .bool_false_representation = "0"};
for (size_t i = 0; i < columns.size(); ++i)
{
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
TypeIndex type_index = data_type->getTypeId();
if (columns[i]->isNullAt(row_num))
{
payload_size += 1;
serialized.emplace_back("\xfb");
}
// Arbitrary precision DateTime64 needs to be forced into precision 6, as it is the maximum that MySQL supports
else if (type_index == TypeIndex::DateTime64)
{
WriteBufferFromOwnString ostr;
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
writeDateTimeText<'-', ':', ' '>(LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))), ostr);
ostr.write('.');
writeDateTime64FractionalText<DateTime64>(components.fractional, 6, ostr);
payload_size += getLengthEncodedStringSize(ostr.str());
serialized.push_back(std::move(ostr.str()));
}
else
{
WriteBufferFromOwnString ostr;
@ -141,7 +157,7 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP
CharacterSet charset = CharacterSet::binary;
int flags = 0;
uint8_t decimals = 0;
DataTypePtr normalized_data_type = removeLowCardinality(removeNullable(data_type));
DataTypePtr normalized_data_type = removeLowCardinalityAndNullable(data_type);
TypeIndex type_index = normalized_data_type->getTypeId();
switch (type_index)
{

View File

@ -76,7 +76,7 @@ protected:
void writePayloadImpl(WriteBuffer & buffer) const override;
public:
ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_);
ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_);
};
class ComFieldList : public LimitedReadPacket

View File

@ -176,4 +176,8 @@ DataTypePtr removeLowCardinality(const DataTypePtr & type)
return type;
}
DataTypePtr removeLowCardinalityAndNullable(const DataTypePtr & type)
{
return removeNullable(removeLowCardinality(type));
};
}

View File

@ -92,4 +92,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column);
/// Convert column of type from_type to type to_type by converting nested LowCardinality columns.
ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
/// Removes LowCardinality and Nullable in a correct order and returns T
/// if the type is LowCardinality(T) or LowCardinality(Nullable(T)); type otherwise
DataTypePtr removeLowCardinalityAndNullable(const DataTypePtr & type);
}

View File

@ -884,7 +884,7 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
{
for (const auto & [name, type] : properties.columns.getAllPhysical())
{
auto basic_type = removeLowCardinality(removeNullable(type));
auto basic_type = removeLowCardinalityAndNullable(type);
if (const auto * fixed_string = typeid_cast<const DataTypeFixedString *>(basic_type.get()))
{
if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS)

View File

@ -48,7 +48,7 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings
if (!settings.allow_suspicious_fixed_string_types)
{
auto basic_type = removeLowCardinality(removeNullable(type));
auto basic_type = removeLowCardinalityAndNullable(type);
if (const auto * fixed_string = typeid_cast<const DataTypeFixedString *>(basic_type.get()))
{
if (fixed_string->getN() > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS)

View File

@ -69,7 +69,7 @@ void MySQLOutputFormat::consume(Chunk chunk)
{
for (size_t i = 0; i < chunk.getNumRows(); ++i)
{
ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), static_cast<int>(i));
ProtocolText::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), static_cast<int>(i));
packet_endpoint->sendPacket(row_packet);
}
}

View File

@ -1,15 +0,0 @@
33jdbcnull
44cknull
0
1
2
3
4
5
6
7
8
9
10
11
12

View File

@ -117,3 +117,13 @@ Row #2
dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0
dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022
### testSuspiciousNullableLowCardinalityTypes
Row #1
f type is FLOAT, value: 1.0
d type is DATE, value: 2022-04-15
dt type is TIMESTAMP, value: 2021-06-04 13:55:11.0
Row #2
f type is FLOAT, value: 3.14
d type is DATE, value: 1970-01-01
dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0

View File

@ -1,4 +1,4 @@
CREATE OR REPLACE TABLE ps_simple_data_types
CREATE OR REPLACE TABLE simple_data_types
(
i8 Int8,
i16 Int16,
@ -17,7 +17,7 @@ CREATE OR REPLACE TABLE ps_simple_data_types
b Boolean
) ENGINE MergeTree ORDER BY i8;
INSERT INTO ps_simple_data_types
INSERT INTO simple_data_types
VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727,
57896044618658097711785492504343953926634992332820282019728792003956564819967,
255, 65535, 4294967295, 18446744073709551615, 340282366920938463463374607431768211455,
@ -28,7 +28,7 @@ VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303
120, 1234, 51234, 421342, 15324355, 41345135123432,
-0.7968956, -0.113259, TRUE);
CREATE OR REPLACE TABLE ps_string_types
CREATE OR REPLACE TABLE string_types
(
s String,
sn Nullable(String),
@ -36,22 +36,22 @@ CREATE OR REPLACE TABLE ps_string_types
nlc LowCardinality(Nullable(String))
) ENGINE MergeTree ORDER BY s;
INSERT INTO ps_string_types
INSERT INTO string_types
VALUES ('foo', 'bar', 'qaz', 'qux'),
('42', NULL, 'test', NULL);
CREATE OR REPLACE TABLE ps_low_cardinality_and_nullable_types
CREATE OR REPLACE TABLE low_cardinality_and_nullable_types
(
ilc LowCardinality(Int32),
dlc LowCardinality(Date),
ni Nullable(Int32)
) ENGINE MergeTree ORDER BY ilc;
INSERT INTO ps_low_cardinality_and_nullable_types
INSERT INTO low_cardinality_and_nullable_types
VALUES (42, '2011-02-05', NULL),
(-54, '1970-01-01', 144);
CREATE OR REPLACE TABLE ps_decimal_types
CREATE OR REPLACE TABLE decimal_types
(
d32 Decimal(9, 2),
d64 Decimal(18, 3),
@ -60,7 +60,7 @@ CREATE OR REPLACE TABLE ps_decimal_types
d256 Decimal(76, 20)
) ENGINE MergeTree ORDER BY d32;
INSERT INTO ps_decimal_types
INSERT INTO decimal_types
VALUES (1234567.89,
123456789123456.789,
12345678912345678912.1234567891,
@ -68,7 +68,7 @@ VALUES (1234567.89,
12345678912345678912345678911234567891234567891234567891.12345678911234567891),
(-1.55, 6.03, 5, -1224124.23423, -54342.3);
CREATE OR REPLACE TABLE ps_misc_types
CREATE OR REPLACE TABLE misc_types
(
a Array(String),
u UUID,
@ -76,10 +76,10 @@ CREATE OR REPLACE TABLE ps_misc_types
m Map(String, Int32)
) ENGINE MergeTree ORDER BY u;
INSERT INTO ps_misc_types
INSERT INTO misc_types
VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255});
CREATE OR REPLACE TABLE ps_date_types
CREATE OR REPLACE TABLE date_types
(
d Date,
d32 Date32,
@ -89,7 +89,7 @@ CREATE OR REPLACE TABLE ps_date_types
dt64_9 DateTime64(9, 'UTC')
) ENGINE MergeTree ORDER BY d;
INSERT INTO ps_date_types
INSERT INTO date_types
VALUES ('2149-06-06', '2178-04-16', '2106-02-07 06:28:15',
'2106-02-07 06:28:15.123',
'2106-02-07 06:28:15.123456',
@ -97,9 +97,9 @@ VALUES ('2149-06-06', '2178-04-16', '2106-02-07 06:28:15',
('1970-01-01', '1900-01-01', '1970-01-01 00:00:00',
'1900-01-01 00:00:00.001',
'1900-01-01 00:00:00.000001',
'1900-01-01 00:00:00.000000001');;
'1900-01-01 00:00:00.000000001');
CREATE OR REPLACE TABLE ps_unusual_datetime64_scales
CREATE OR REPLACE TABLE unusual_datetime64_scales
(
dt64_0 DateTime64(0, 'UTC'),
dt64_1 DateTime64(1, 'UTC'),
@ -110,7 +110,7 @@ CREATE OR REPLACE TABLE ps_unusual_datetime64_scales
dt64_8 DateTime64(8, 'UTC')
) ENGINE MergeTree ORDER BY dt64_0;
INSERT INTO ps_unusual_datetime64_scales
INSERT INTO unusual_datetime64_scales
VALUES ('2022-04-13 03:17:45',
'2022-04-13 03:17:45.1',
'2022-04-13 03:17:45.12',
@ -126,12 +126,23 @@ VALUES ('2022-04-13 03:17:45',
'2022-04-13 03:17:45.0000001',
'2022-04-13 03:17:45.00000001');
CREATE OR REPLACE TABLE ps_datetime_timezones
CREATE OR REPLACE TABLE datetime_timezones
(
dt DateTime('Europe/Amsterdam'),
dt64_3 DateTime64(3, 'Asia/Shanghai')
) ENGINE MergeTree ORDER BY dt;
INSERT INTO ps_datetime_timezones
INSERT INTO datetime_timezones
VALUES ('2022-09-04 20:31:05', '2022-09-04 20:31:05.022'),
('1970-01-01 01:00:00', '1969-12-31 16:00:00');
CREATE OR REPLACE TABLE suspicious_nullable_low_cardinality_types
(
f LowCardinality(Nullable(Float32)),
d LowCardinality(Nullable(Date)),
dt LowCardinality(Nullable(DateTime('UTC')))
) ENGINE MergeTree ORDER BY tuple();
INSERT INTO suspicious_nullable_low_cardinality_types
VALUES (1, '2022-04-15', '2021-06-04 13:55:11'),
(3.14, '1970-01-01', '1970-01-01 00:00:00');

View File

@ -0,0 +1,129 @@
### testSimpleDataTypes
Row #1
i8 type is TINYINT, value: -128
i16 type is SMALLINT, value: -32768
i32 type is INT, value: -2147483648
i64 type is BIGINT, value: -9223372036854775808
i128 type is CHAR, value: -170141183460469231731687303715884105728
i256 type is CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968
ui8 type is TINYINT, value: 120
ui16 type is SMALLINT, value: 1234
ui32 type is INT, value: 51234
ui64 type is BIGINT, value: 421342
ui128 type is CHAR, value: 15324355
ui256 type is CHAR, value: 41345135123432
f32 type is FLOAT, value: -0.796896
f64 type is DOUBLE, value: -0.113259
b type is TINYINT, value: true
Row #2
i8 type is TINYINT, value: 127
i16 type is SMALLINT, value: 32767
i32 type is INT, value: 2147483647
i64 type is BIGINT, value: 9223372036854775807
i128 type is CHAR, value: 170141183460469231731687303715884105727
i256 type is CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967
ui8 type is TINYINT, value: 255
ui16 type is SMALLINT, value: 65535
ui32 type is INT, value: 4294967295
ui64 type is BIGINT, value: 18446744073709551615
ui128 type is CHAR, value: 340282366920938463463374607431768211455
ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935
f32 type is FLOAT, value: 1.234000
f64 type is DOUBLE, value: 3.352451
b type is TINYINT, value: false
### testStringTypes
Row #1
s type is CHAR, value: 42
sn type is CHAR, value: null
lc type is CHAR, value: test
nlc type is CHAR, value: null
Row #2
s type is CHAR, value: foo
sn type is CHAR, value: bar
lc type is CHAR, value: qaz
nlc type is CHAR, value: qux
### testLowCardinalityAndNullableTypes
Row #1
ilc type is INT, value: -54
dlc type is DATE, value: 1970-01-01
ni type is INT, value: 144
Row #2
ilc type is INT, value: 42
dlc type is DATE, value: 2011-02-05
ni type is INT, value: 0
### testDecimalTypes
Row #1
d32 type is DECIMAL, value: -1.55
d64 type is DECIMAL, value: 6.03
d128_native type is DECIMAL, value: 5
d128_text type is CHAR, value: -1224124.23423
d256 type is CHAR, value: -54342.3
Row #2
d32 type is DECIMAL, value: 1234567.89
d64 type is DECIMAL, value: 123456789123456.789
d128_native type is DECIMAL, value: 12345678912345678912.1234567891
d128_text type is CHAR, value: 1234567.8912345678912345678911234567891
d256 type is CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891
### testMiscTypes
Row #1
a type is CHAR, value: ['foo','bar']
u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3
t type is CHAR, value: (42,'qaz')
m type is CHAR, value: {'qux':144,'text':255}
### testDateTypes
Row #1
d type is DATE, value: 1970-01-01
d32 type is DATE, value: 1900-01-01
dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0
dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.001
dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001
dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0
Row #2
d type is DATE, value: 2149-06-06
d32 type is DATE, value: 2178-04-16
dt type is TIMESTAMP, value: 2106-02-07 06:28:15.0
dt64_3 type is TIMESTAMP, value: 2106-02-07 06:28:15.123
dt64_6 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456
dt64_9 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456
### testUnusualDateTime64Scales
Row #1
dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0
dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1
dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.12
dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.1234
dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.12345
dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456
dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456
Row #2
dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0
dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1
dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.01
dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.0001
dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.00001
dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.0
dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.0
### testDateTimeTimezones
Row #1
dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0
dt64_3 type is TIMESTAMP, value: 1969-12-31 16:00:00.0
Row #2
dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0
dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022
### testSuspiciousNullableLowCardinalityTypes
Row #1
f type is FLOAT, value: 1.0
d type is DATE, value: 2022-04-15
dt type is TIMESTAMP, value: 2021-06-04 13:55:11.0
Row #2
f type is FLOAT, value: 3.14
d type is DATE, value: 1970-01-01
dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0

View File

@ -7,6 +7,8 @@ import os
import time
import logging
from typing import Literal
import docker
import pymysql.connections
import pytest
@ -790,79 +792,32 @@ def test_mysqljs_client(started_cluster, nodejs_container):
assert code == 1
def test_java_client(started_cluster, java_container):
with open(os.path.join(SCRIPT_DIR, "java.reference")) as fp:
def test_java_client_text(started_cluster, java_container):
command = setup_java_client(started_cluster, "false")
code, (stdout, stderr) = java_container.exec_run(
command,
demux=True,
)
with open(os.path.join(SCRIPT_DIR, "java_client_text.reference")) as fp:
reference = fp.read()
# database not exists exception.
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database "
"abc".format(host=started_cluster.get_instance_ip("node"), port=server_port),
demux=True,
)
assert code == 1
# empty password passed.
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database "
"default".format(
host=started_cluster.get_instance_ip("node"), port=server_port
),
demux=True,
)
assert code == 0
assert stdout.decode() == reference
assert code == 0
# non-empty password passed.
def test_java_client_binary(started_cluster, java_container):
command = setup_java_client(started_cluster, "true")
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database "
"default".format(
host=started_cluster.get_instance_ip("node"), port=server_port
),
command,
demux=True,
)
assert code == 0
assert stdout.decode() == reference
# double-sha1 password passed.
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database "
"default".format(
host=started_cluster.get_instance_ip("node"), port=server_port
),
demux=True,
)
assert code == 0
assert stdout.decode() == reference
def test_prepared_statements(started_cluster, java_container):
with open(os.path.join(SCRIPT_DIR, "prepared_statements.reference")) as fp:
with open(os.path.join(SCRIPT_DIR, "java_client_binary.reference")) as fp:
reference = fp.read()
with open(os.path.join(SCRIPT_DIR, "prepared_statements_test.sql")) as sql:
statements = list(
filter(
lambda s: s != "",
map(lambda s: s.strip().replace("\n", " "), sql.read().split(";")),
)
)
for statement in statements:
node.query(
statement,
settings={"password": "123", "allow_suspicious_low_cardinality_types": 1},
)
code, (stdout, stderr) = java_container.exec_run(
"java PreparedStatementsTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database "
"default".format(
host=started_cluster.get_instance_ip("node"), port=server_port
),
demux=True,
)
assert code == 0
assert stdout.decode() == reference
assert code == 0
def test_types(started_cluster):
@ -924,3 +879,31 @@ def test_types(started_cluster):
assert math.isnan(result[key])
else:
assert result[key] == value
def setup_java_client(started_cluster, binary: Literal["true", "false"]):
with open(os.path.join(SCRIPT_DIR, "java_client_test.sql")) as sql:
statements = list(
filter(
lambda s: s != "",
map(lambda s: s.strip().replace("\n", " "), sql.read().split(";")),
)
)
for statement in statements:
node.query(
statement,
settings={"password": "123", "allow_suspicious_low_cardinality_types": 1},
)
return (
"java MySQLJavaClientTest "
"--host {host} "
"--port {port} "
"--user user_with_double_sha1 "
"--password abacaba "
"--database default "
"--binary {binary}"
).format(
host=started_cluster.get_instance_ip("node"), port=server_port, binary=binary
)