mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix LowCardinality/Nullable columns issues
This commit is contained in:
parent
c0961d9378
commit
44e1f6b53d
@ -43,6 +43,7 @@ public class PreparedStatementsTest {
|
|||||||
Connection conn = DriverManager.getConnection(jdbcUrl, user, password);
|
Connection conn = DriverManager.getConnection(jdbcUrl, user, password);
|
||||||
testSimpleDataTypes(conn);
|
testSimpleDataTypes(conn);
|
||||||
testStringTypes(conn);
|
testStringTypes(conn);
|
||||||
|
testLowCardinalityAndNullableTypes(conn);
|
||||||
testDecimalTypes(conn);
|
testDecimalTypes(conn);
|
||||||
testMiscTypes(conn);
|
testMiscTypes(conn);
|
||||||
testDateTypes(conn);
|
testDateTypes(conn);
|
||||||
@ -94,6 +95,20 @@ public class PreparedStatementsTest {
|
|||||||
System.out.println();
|
System.out.println();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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();
|
||||||
|
int rowNum = 1;
|
||||||
|
while (rs.next()) {
|
||||||
|
System.out.printf("Row #%d\n", rowNum++);
|
||||||
|
System.out.printf("%s, value: %s\n", getMysqlType(rs, "ilc"), rs.getInt("ilc"));
|
||||||
|
System.out.printf("%s, value: %s\n", getMysqlType(rs, "dlc"), rs.getDate("dlc"));
|
||||||
|
// NULL int is represented as zero
|
||||||
|
System.out.printf("%s, value: %s\n", getMysqlType(rs, "ni"), rs.getInt("ni"));
|
||||||
|
}
|
||||||
|
System.out.println();
|
||||||
|
}
|
||||||
|
|
||||||
private static void testDecimalTypes(Connection conn) throws SQLException {
|
private static void testDecimalTypes(Connection conn) throws SQLException {
|
||||||
System.out.println("### testDecimalTypes");
|
System.out.println("### testDecimalTypes");
|
||||||
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery();
|
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery();
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include "Common/LocalDate.h"
|
#include "Common/LocalDate.h"
|
||||||
#include "Common/LocalDateTime.h"
|
#include "Common/LocalDateTime.h"
|
||||||
#include "Columns/ColumnLowCardinality.h"
|
#include "Columns/ColumnLowCardinality.h"
|
||||||
|
#include "Columns/ColumnNullable.h"
|
||||||
#include "Columns/ColumnVector.h"
|
#include "Columns/ColumnVector.h"
|
||||||
#include "Columns/ColumnsDateTime.h"
|
#include "Columns/ColumnsDateTime.h"
|
||||||
#include "Core/DecimalFunctions.h"
|
#include "Core/DecimalFunctions.h"
|
||||||
@ -31,7 +32,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
|
|||||||
FormatSettings format_settings;
|
FormatSettings format_settings;
|
||||||
for (size_t i = 0; i < columns.size(); ++i)
|
for (size_t i = 0; i < columns.size(); ++i)
|
||||||
{
|
{
|
||||||
ColumnPtr col = columns[i];
|
ColumnPtr col = getColumn(i);
|
||||||
if (col->isNullAt(row_num))
|
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
|
// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row
|
||||||
@ -152,7 +153,7 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
|
|||||||
buffer.write(null_bitmap.data(), null_bitmap_size);
|
buffer.write(null_bitmap.data(), null_bitmap_size);
|
||||||
for (size_t i = 0; i < columns.size(); ++i)
|
for (size_t i = 0; i < columns.size(); ++i)
|
||||||
{
|
{
|
||||||
ColumnPtr col = columns[i];
|
ColumnPtr col = getColumn(i);
|
||||||
if (col->isNullAt(row_num))
|
if (col->isNullAt(row_num))
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
@ -351,6 +352,14 @@ ResultSetRow::DateTime64ComponentsWithScale ResultSetRow::getDateTime64Component
|
|||||||
|
|
||||||
return {components, scale};
|
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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,6 +20,7 @@ class ResultSetRow : public IMySQLWritePacket
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
DateTime64ComponentsWithScale getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const;
|
DateTime64ComponentsWithScale getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const;
|
||||||
|
ColumnPtr getColumn(size_t i) const;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
int row_num;
|
int row_num;
|
||||||
|
@ -4,6 +4,8 @@
|
|||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include "Common/assert_cast.h"
|
#include "Common/assert_cast.h"
|
||||||
#include "Core/MySQL/IMySQLWritePacket.h"
|
#include "Core/MySQL/IMySQLWritePacket.h"
|
||||||
|
#include "DataTypes/DataTypeLowCardinality.h"
|
||||||
|
#include "DataTypes/DataTypeNullable.h"
|
||||||
#include "DataTypes/DataTypesDecimal.h"
|
#include "DataTypes/DataTypesDecimal.h"
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -140,7 +142,7 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP
|
|||||||
CharacterSet charset = CharacterSet::binary;
|
CharacterSet charset = CharacterSet::binary;
|
||||||
int flags = 0;
|
int flags = 0;
|
||||||
uint8_t decimals = 0;
|
uint8_t decimals = 0;
|
||||||
TypeIndex type_index = data_type->getTypeId();
|
TypeIndex type_index = removeLowCardinality(removeNullable(data_type))->getTypeId();
|
||||||
switch (type_index)
|
switch (type_index)
|
||||||
{
|
{
|
||||||
case TypeIndex::UInt8:
|
case TypeIndex::UInt8:
|
||||||
|
@ -404,7 +404,7 @@ void MySQLHandler::comStmtExecute(ReadBuffer & payload)
|
|||||||
|
|
||||||
auto statement_opt = getPreparedStatement(statement_id);
|
auto statement_opt = getPreparedStatement(statement_id);
|
||||||
if (statement_opt.has_value())
|
if (statement_opt.has_value())
|
||||||
MySQLHandler::comQuery(statement_opt.value().get(), true);
|
MySQLHandler::comQuery(statement_opt.value(), true);
|
||||||
else
|
else
|
||||||
packet_endpoint->sendPacket(ERRPacket(), true);
|
packet_endpoint->sendPacket(ERRPacket(), true);
|
||||||
};
|
};
|
||||||
@ -428,7 +428,7 @@ std::optional<UInt32> MySQLHandler::emplacePreparedStatement(String statement)
|
|||||||
LOG_ERROR(log, "Too many prepared statements");
|
LOG_ERROR(log, "Too many prepared statements");
|
||||||
current_prepared_statement_id = 0;
|
current_prepared_statement_id = 0;
|
||||||
prepared_statements.clear();
|
prepared_statements.clear();
|
||||||
return std::nullopt;
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
uint32_t statement_id = current_prepared_statement_id;
|
uint32_t statement_id = current_prepared_statement_id;
|
||||||
@ -444,24 +444,23 @@ std::optional<UInt32> MySQLHandler::emplacePreparedStatement(String statement)
|
|||||||
statement,
|
statement,
|
||||||
statement_id,
|
statement_id,
|
||||||
prepared_statements.at(statement_id));
|
prepared_statements.at(statement_id));
|
||||||
return std::nullopt;
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
prepared_statements.emplace(statement_id, statement);
|
prepared_statements.emplace(statement_id, statement);
|
||||||
return std::make_optional(statement_id);
|
return std::make_optional(statement_id);
|
||||||
};
|
};
|
||||||
|
|
||||||
std::optional<std::reference_wrapper<ReadBufferFromString>> MySQLHandler::getPreparedStatement(UInt32 statement_id)
|
std::optional<ReadBufferFromString> MySQLHandler::getPreparedStatement(UInt32 statement_id)
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> lock(prepared_statements_mutex);
|
std::lock_guard<std::mutex> lock(prepared_statements_mutex);
|
||||||
if (!prepared_statements.contains(statement_id))
|
if (!prepared_statements.contains(statement_id))
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Could not find prepared statement with id {}", statement_id);
|
LOG_ERROR(log, "Could not find prepared statement with id {}", statement_id);
|
||||||
return std::nullopt;
|
return {};
|
||||||
}
|
}
|
||||||
// Temporary workaround as we work only with queries that do not bind any parameters atm
|
// Temporary workaround as we work only with queries that do not bind any parameters atm
|
||||||
ReadBufferFromString statement(prepared_statements.at(statement_id));
|
return std::make_optional<ReadBufferFromString>(prepared_statements.at(statement_id));
|
||||||
return std::make_optional(std::reference_wrapper(statement));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void MySQLHandler::erasePreparedStatement(UInt32 statement_id)
|
void MySQLHandler::erasePreparedStatement(UInt32 statement_id)
|
||||||
|
@ -71,7 +71,7 @@ protected:
|
|||||||
/// Contains statement_id if the statement was emplaced successfully
|
/// Contains statement_id if the statement was emplaced successfully
|
||||||
std::optional<UInt32> emplacePreparedStatement(String statement);
|
std::optional<UInt32> emplacePreparedStatement(String statement);
|
||||||
/// Contains statement as a buffer if we could find previously stored statement using provided statement_id
|
/// Contains statement as a buffer if we could find previously stored statement using provided statement_id
|
||||||
std::optional<std::reference_wrapper<ReadBufferFromString>> getPreparedStatement(UInt32 statement_id);
|
std::optional<ReadBufferFromString> getPreparedStatement(UInt32 statement_id);
|
||||||
void erasePreparedStatement(UInt32 statement_id);
|
void erasePreparedStatement(UInt32 statement_id);
|
||||||
|
|
||||||
virtual void authPluginSSL();
|
virtual void authPluginSSL();
|
||||||
|
@ -35,15 +35,25 @@ b type is BIT, value: false
|
|||||||
### testStringTypes
|
### testStringTypes
|
||||||
Row #1
|
Row #1
|
||||||
s type is CHAR, value: 42
|
s type is CHAR, value: 42
|
||||||
sn type is CHAR, value: null
|
sn type is CHAR, value: ᴺᵁᴸᴸ
|
||||||
lc type is CHAR, value: test
|
lc type is CHAR, value: test
|
||||||
nlc type is CHAR, value: null
|
nlc type is CHAR, value: ᴺᵁᴸᴸ
|
||||||
Row #2
|
Row #2
|
||||||
s type is CHAR, value: foo
|
s type is CHAR, value: foo
|
||||||
sn type is CHAR, value: bar
|
sn type is CHAR, value: bar
|
||||||
lc type is CHAR, value: qaz
|
lc type is CHAR, value: qaz
|
||||||
nlc type is CHAR, value: qux
|
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
|
### testDecimalTypes
|
||||||
Row #1
|
Row #1
|
||||||
d32 type is DECIMAL, value: -1.55
|
d32 type is DECIMAL, value: -1.55
|
||||||
|
@ -40,6 +40,17 @@ INSERT INTO ps_string_types
|
|||||||
VALUES ('foo', 'bar', 'qaz', 'qux'),
|
VALUES ('foo', 'bar', 'qaz', 'qux'),
|
||||||
('42', NULL, 'test', NULL);
|
('42', NULL, 'test', NULL);
|
||||||
|
|
||||||
|
CREATE OR REPLACE TABLE ps_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
|
||||||
|
VALUES (42, '2011-02-05', NULL),
|
||||||
|
(-54, '1970-01-01', 144);
|
||||||
|
|
||||||
CREATE OR REPLACE TABLE ps_decimal_types
|
CREATE OR REPLACE TABLE ps_decimal_types
|
||||||
(
|
(
|
||||||
d32 Decimal(9, 2),
|
d32 Decimal(9, 2),
|
||||||
|
@ -818,7 +818,10 @@ def test_prepared_statements(started_cluster, java_container):
|
|||||||
)
|
)
|
||||||
|
|
||||||
for statement in statements:
|
for statement in statements:
|
||||||
node.query(statement, settings={"password": "123"})
|
node.query(
|
||||||
|
statement,
|
||||||
|
settings={"password": "123", "allow_suspicious_low_cardinality_types": 1},
|
||||||
|
)
|
||||||
|
|
||||||
code, (stdout, stderr) = java_container.exec_run(
|
code, (stdout, stderr) = java_container.exec_run(
|
||||||
"java PreparedStatementsTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database "
|
"java PreparedStatementsTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database "
|
||||||
|
@ -4,363 +4,363 @@ Create pseudo-random database name
|
|||||||
Create tab duplicate table
|
Create tab duplicate table
|
||||||
Run MySQL test
|
Run MySQL test
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function AggregateFunction(sum, Int32) 0 NULL
|
aggregate_function AggregateFunction(sum, Int32) NO NULL
|
||||||
array_value Array(Int32) 0 NULL
|
array_value Array(Int32) NO NULL
|
||||||
boolean_value UInt8 0 NULL
|
boolean_value UInt8 NO NULL
|
||||||
date32_value Date32 0 NULL
|
date32_value Date32 NO NULL
|
||||||
date_value Date 0 NULL
|
date_value Date NO NULL
|
||||||
datetime64_value DateTime64(3) 0 NULL
|
datetime64_value DateTime64(3) NO NULL
|
||||||
datetime_value DateTime 0 NULL
|
datetime_value DateTime NO NULL
|
||||||
decimal_value Decimal(10, 2) 0 NULL
|
decimal_value Decimal(10, 2) NO NULL
|
||||||
enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3) 0 NULL
|
enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3) NO NULL
|
||||||
fixed_string_value FixedString(10) 0 NULL
|
fixed_string_value FixedString(10) NO NULL
|
||||||
float32 Float32 0 NULL
|
float32 Float32 NO NULL
|
||||||
float64 Float64 0 NULL
|
float64 Float64 NO NULL
|
||||||
int128 Int128 0 NULL
|
int128 Int128 NO NULL
|
||||||
int16 Int16 0 NULL
|
int16 Int16 NO NULL
|
||||||
int256 Int256 0 NULL
|
int256 Int256 NO NULL
|
||||||
int32 Int32 0 NULL
|
int32 Int32 NO NULL
|
||||||
int64 Int64 0 NULL
|
int64 Int64 NO NULL
|
||||||
int8 Int8 0 NULL
|
int8 Int8 NO NULL
|
||||||
ipv4_value IPv4 0 NULL
|
ipv4_value IPv4 NO NULL
|
||||||
ipv6_value IPv6 0 NULL
|
ipv6_value IPv6 NO NULL
|
||||||
json_value Object('json') 0 NULL
|
json_value Object('json') NO NULL
|
||||||
low_cardinality LowCardinality(String) 0 NULL
|
low_cardinality LowCardinality(String) NO NULL
|
||||||
low_cardinality_date LowCardinality(DateTime) 0 NULL
|
low_cardinality_date LowCardinality(DateTime) NO NULL
|
||||||
map_value Map(String, Int32) 0 NULL
|
map_value Map(String, Int32) NO NULL
|
||||||
nested.nested_int Array(Int32) 0 NULL
|
nested.nested_int Array(Int32) NO NULL
|
||||||
nested.nested_string Array(String) 0 NULL
|
nested.nested_string Array(String) NO NULL
|
||||||
nint32 Nullable(Int32) 1 NULL
|
nint32 Nullable(Int32) YES NULL
|
||||||
nullable_value Nullable(Int32) 1 NULL
|
nullable_value Nullable(Int32) YES NULL
|
||||||
string_value String 0 NULL
|
string_value String NO NULL
|
||||||
tuple_value Tuple(Int32, String) 0 NULL
|
tuple_value Tuple(Int32, String) NO NULL
|
||||||
uint128 UInt128 0 NULL
|
uint128 UInt128 NO NULL
|
||||||
uint16 UInt16 0 NULL
|
uint16 UInt16 NO NULL
|
||||||
uint256 UInt256 0 NULL
|
uint256 UInt256 NO NULL
|
||||||
uint32 UInt32 0 NULL
|
uint32 UInt32 NO NULL
|
||||||
uint64 UInt64 0 PRI SOR NULL
|
uint64 UInt64 NO PRI SOR NULL
|
||||||
uint8 UInt8 0 NULL
|
uint8 UInt8 NO NULL
|
||||||
uuid_value UUID 0 NULL
|
uuid_value UUID NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra collation comment privileges
|
field type null key default extra collation comment privileges
|
||||||
aggregate_function TEXT 0 NULL NULL
|
aggregate_function TEXT NO NULL NULL
|
||||||
array_value TEXT 0 NULL NULL
|
array_value TEXT NO NULL NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL NULL
|
boolean_value TINYINT UNSIGNED NO NULL NULL
|
||||||
date32_value DATE 0 NULL NULL
|
date32_value DATE NO NULL NULL
|
||||||
date_value DATE 0 NULL NULL
|
date_value DATE NO NULL NULL
|
||||||
datetime64_value DATETIME 0 NULL NULL
|
datetime64_value DATETIME NO NULL NULL
|
||||||
datetime_value DATETIME 0 NULL NULL
|
datetime_value DATETIME NO NULL NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL NULL
|
decimal_value DECIMAL(10, 2) NO NULL NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL NULL
|
||||||
fixed_string_value TEXT 0 NULL NULL
|
fixed_string_value TEXT NO NULL NULL
|
||||||
float32 FLOAT 0 NULL NULL
|
float32 FLOAT NO NULL NULL
|
||||||
float64 DOUBLE 0 NULL NULL
|
float64 DOUBLE NO NULL NULL
|
||||||
int128 TEXT 0 NULL NULL
|
int128 TEXT NO NULL NULL
|
||||||
int16 SMALLINT 0 NULL NULL
|
int16 SMALLINT NO NULL NULL
|
||||||
int256 TEXT 0 NULL NULL
|
int256 TEXT NO NULL NULL
|
||||||
int32 INTEGER 0 NULL NULL
|
int32 INTEGER NO NULL NULL
|
||||||
int64 BIGINT 0 NULL NULL
|
int64 BIGINT NO NULL NULL
|
||||||
int8 TINYINT 0 NULL NULL
|
int8 TINYINT NO NULL NULL
|
||||||
ipv4_value TEXT 0 NULL NULL
|
ipv4_value TEXT NO NULL NULL
|
||||||
ipv6_value TEXT 0 NULL NULL
|
ipv6_value TEXT NO NULL NULL
|
||||||
json_value JSON 0 NULL NULL
|
json_value JSON NO NULL NULL
|
||||||
low_cardinality BLOB 0 NULL NULL
|
low_cardinality BLOB NO NULL NULL
|
||||||
low_cardinality_date DATETIME 0 NULL NULL
|
low_cardinality_date DATETIME NO NULL NULL
|
||||||
map_value JSON 0 NULL NULL
|
map_value JSON NO NULL NULL
|
||||||
nested.nested_int TEXT 0 NULL NULL
|
nested.nested_int TEXT NO NULL NULL
|
||||||
nested.nested_string TEXT 0 NULL NULL
|
nested.nested_string TEXT NO NULL NULL
|
||||||
nint32 INTEGER 0 NULL NULL
|
nint32 INTEGER NO NULL NULL
|
||||||
nullable_value INTEGER 0 NULL NULL
|
nullable_value INTEGER NO NULL NULL
|
||||||
string_value BLOB 0 NULL NULL
|
string_value BLOB NO NULL NULL
|
||||||
tuple_value JSON 0 NULL NULL
|
tuple_value JSON NO NULL NULL
|
||||||
uint128 TEXT 0 NULL NULL
|
uint128 TEXT NO NULL NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL NULL
|
uint16 SMALLINT UNSIGNED NO NULL NULL
|
||||||
uint256 TEXT 0 NULL NULL
|
uint256 TEXT NO NULL NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL NULL
|
uint32 INTEGER UNSIGNED NO NULL NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL NULL
|
uint8 TINYINT UNSIGNED NO NULL NULL
|
||||||
uuid_value CHAR 0 NULL NULL
|
uuid_value CHAR NO NULL NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
field type null key default extra
|
field type null key default extra
|
||||||
aggregate_function TEXT 0 NULL
|
aggregate_function TEXT NO NULL
|
||||||
array_value TEXT 0 NULL
|
array_value TEXT NO NULL
|
||||||
boolean_value TINYINT UNSIGNED 0 NULL
|
boolean_value TINYINT UNSIGNED NO NULL
|
||||||
date32_value DATE 0 NULL
|
date32_value DATE NO NULL
|
||||||
date_value DATE 0 NULL
|
date_value DATE NO NULL
|
||||||
datetime64_value DATETIME 0 NULL
|
datetime64_value DATETIME NO NULL
|
||||||
datetime_value DATETIME 0 NULL
|
datetime_value DATETIME NO NULL
|
||||||
decimal_value DECIMAL(10, 2) 0 NULL
|
decimal_value DECIMAL(10, 2) NO NULL
|
||||||
enum_value ENUM('apple', 'banana', 'orange') 0 NULL
|
enum_value ENUM('apple', 'banana', 'orange') NO NULL
|
||||||
fixed_string_value TEXT 0 NULL
|
fixed_string_value TEXT NO NULL
|
||||||
float32 FLOAT 0 NULL
|
float32 FLOAT NO NULL
|
||||||
float64 DOUBLE 0 NULL
|
float64 DOUBLE NO NULL
|
||||||
int128 TEXT 0 NULL
|
int128 TEXT NO NULL
|
||||||
int16 SMALLINT 0 NULL
|
int16 SMALLINT NO NULL
|
||||||
int256 TEXT 0 NULL
|
int256 TEXT NO NULL
|
||||||
int32 INTEGER 0 NULL
|
int32 INTEGER NO NULL
|
||||||
int64 BIGINT 0 NULL
|
int64 BIGINT NO NULL
|
||||||
int8 TINYINT 0 NULL
|
int8 TINYINT NO NULL
|
||||||
ipv4_value TEXT 0 NULL
|
ipv4_value TEXT NO NULL
|
||||||
ipv6_value TEXT 0 NULL
|
ipv6_value TEXT NO NULL
|
||||||
json_value JSON 0 NULL
|
json_value JSON NO NULL
|
||||||
low_cardinality BLOB 0 NULL
|
low_cardinality BLOB NO NULL
|
||||||
low_cardinality_date DATETIME 0 NULL
|
low_cardinality_date DATETIME NO NULL
|
||||||
map_value JSON 0 NULL
|
map_value JSON NO NULL
|
||||||
nested.nested_int TEXT 0 NULL
|
nested.nested_int TEXT NO NULL
|
||||||
nested.nested_string TEXT 0 NULL
|
nested.nested_string TEXT NO NULL
|
||||||
nint32 INTEGER 0 NULL
|
nint32 INTEGER NO NULL
|
||||||
nullable_value INTEGER 0 NULL
|
nullable_value INTEGER NO NULL
|
||||||
string_value BLOB 0 NULL
|
string_value BLOB NO NULL
|
||||||
tuple_value JSON 0 NULL
|
tuple_value JSON NO NULL
|
||||||
uint128 TEXT 0 NULL
|
uint128 TEXT NO NULL
|
||||||
uint16 SMALLINT UNSIGNED 0 NULL
|
uint16 SMALLINT UNSIGNED NO NULL
|
||||||
uint256 TEXT 0 NULL
|
uint256 TEXT NO NULL
|
||||||
uint32 INTEGER UNSIGNED 0 NULL
|
uint32 INTEGER UNSIGNED NO NULL
|
||||||
uint64 BIGINT UNSIGNED 0 PRI SOR NULL
|
uint64 BIGINT UNSIGNED NO PRI SOR NULL
|
||||||
uint8 TINYINT UNSIGNED 0 NULL
|
uint8 TINYINT UNSIGNED NO NULL
|
||||||
uuid_value CHAR 0 NULL
|
uuid_value CHAR NO NULL
|
||||||
|
Loading…
Reference in New Issue
Block a user