mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #35735 from Avogar/allow-read-bools-as-numbers
Allow to infer and parse bools as numbers in JSON input formats
This commit is contained in:
commit
f3f8f27db5
@ -638,6 +638,7 @@ class IColumn;
|
|||||||
M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \
|
M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \
|
||||||
M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \
|
M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \
|
||||||
M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \
|
M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \
|
||||||
|
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
|
||||||
\
|
\
|
||||||
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \
|
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \
|
||||||
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
|
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
|
||||||
|
@ -43,7 +43,7 @@ void SerializationNumber<T>::serializeTextJSON(const IColumn & column, size_t ro
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
void SerializationNumber<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
void SerializationNumber<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||||
{
|
{
|
||||||
bool has_quote = false;
|
bool has_quote = false;
|
||||||
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
|
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
|
||||||
@ -67,7 +67,7 @@ void SerializationNumber<T>::deserializeTextJSON(IColumn & column, ReadBuffer &
|
|||||||
static constexpr bool is_uint8 = std::is_same_v<T, UInt8>;
|
static constexpr bool is_uint8 = std::is_same_v<T, UInt8>;
|
||||||
static constexpr bool is_int8 = std::is_same_v<T, Int8>;
|
static constexpr bool is_int8 = std::is_same_v<T, Int8>;
|
||||||
|
|
||||||
if (is_uint8 || is_int8)
|
if (settings.json.read_bools_as_numbers || is_uint8 || is_int8)
|
||||||
{
|
{
|
||||||
// extra conditions to parse true/false strings into 1/0
|
// extra conditions to parse true/false strings into 1/0
|
||||||
if (istr.eof())
|
if (istr.eof())
|
||||||
|
@ -90,6 +90,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
|
|||||||
format_settings.json.named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects;
|
format_settings.json.named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects;
|
||||||
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
|
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
|
||||||
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
|
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
|
||||||
|
format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers;
|
||||||
format_settings.null_as_default = settings.input_format_null_as_default;
|
format_settings.null_as_default = settings.input_format_null_as_default;
|
||||||
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
|
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
|
||||||
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
|
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
|
||||||
|
@ -133,6 +133,7 @@ struct FormatSettings
|
|||||||
bool escape_forward_slashes = true;
|
bool escape_forward_slashes = true;
|
||||||
bool named_tuples_as_objects = false;
|
bool named_tuples_as_objects = false;
|
||||||
bool serialize_as_strings = false;
|
bool serialize_as_strings = false;
|
||||||
|
bool read_bools_as_numbers = true;
|
||||||
} json;
|
} json;
|
||||||
|
|
||||||
struct
|
struct
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypeMap.h>
|
#include <DataTypes/DataTypeMap.h>
|
||||||
#include <DataTypes/DataTypeObject.h>
|
#include <DataTypes/DataTypeObject.h>
|
||||||
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <Common/JSONParsers/SimdJSONParser.h>
|
#include <Common/JSONParsers/SimdJSONParser.h>
|
||||||
#include <Common/JSONParsers/RapidJSONParser.h>
|
#include <Common/JSONParsers/RapidJSONParser.h>
|
||||||
#include <Common/JSONParsers/DummyJSONParser.h>
|
#include <Common/JSONParsers/DummyJSONParser.h>
|
||||||
@ -118,7 +119,7 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field)
|
|||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
if (field.isBool())
|
if (field.isBool())
|
||||||
return makeNullable(std::make_shared<DataTypeUInt8>());
|
return DataTypeFactory::instance().get("Nullable(Bool)");
|
||||||
|
|
||||||
if (field.isInt64() || field.isUInt64() || field.isDouble())
|
if (field.isInt64() || field.isUInt64() || field.isDouble())
|
||||||
return makeNullable(std::make_shared<DataTypeFloat64>());
|
return makeNullable(std::make_shared<DataTypeFloat64>());
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <Processors/Formats/ISchemaReader.h>
|
#include <Processors/Formats/ISchemaReader.h>
|
||||||
#include <Formats/ReadSchemaUtils.h>
|
#include <Formats/ReadSchemaUtils.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -10,8 +11,63 @@ namespace ErrorCodes
|
|||||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||||
}
|
}
|
||||||
|
|
||||||
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_)
|
static void chooseResultType(
|
||||||
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_)
|
DataTypePtr & type,
|
||||||
|
const DataTypePtr & new_type,
|
||||||
|
bool allow_bools_as_numbers,
|
||||||
|
const DataTypePtr & default_type,
|
||||||
|
const String & column_name,
|
||||||
|
size_t row)
|
||||||
|
{
|
||||||
|
if (!type)
|
||||||
|
type = new_type;
|
||||||
|
|
||||||
|
/// If the new type and the previous type for this column are different,
|
||||||
|
/// we will use default type if we have it or throw an exception.
|
||||||
|
if (new_type && !type->equals(*new_type))
|
||||||
|
{
|
||||||
|
/// Check if we have Bool and Number and if allow_bools_as_numbers
|
||||||
|
/// is true make the result type Number
|
||||||
|
auto not_nullable_type = removeNullable(type);
|
||||||
|
auto not_nullable_new_type = removeNullable(new_type);
|
||||||
|
bool bool_type_presents = isBool(not_nullable_type) || isBool(not_nullable_new_type);
|
||||||
|
bool number_type_presents = isNumber(not_nullable_type) || isNumber(not_nullable_new_type);
|
||||||
|
if (allow_bools_as_numbers && bool_type_presents && number_type_presents)
|
||||||
|
{
|
||||||
|
if (isBool(not_nullable_type))
|
||||||
|
type = new_type;
|
||||||
|
}
|
||||||
|
else if (default_type)
|
||||||
|
type = default_type;
|
||||||
|
else
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
||||||
|
"Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}",
|
||||||
|
type->getName(),
|
||||||
|
column_name,
|
||||||
|
row,
|
||||||
|
new_type->getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void checkTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t max_rows_to_read)
|
||||||
|
{
|
||||||
|
if (!type)
|
||||||
|
{
|
||||||
|
if (!default_type)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
||||||
|
"Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum "
|
||||||
|
"number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference",
|
||||||
|
max_rows_to_read);
|
||||||
|
|
||||||
|
type = default_type;
|
||||||
|
}
|
||||||
|
result.emplace_back(name, type);
|
||||||
|
}
|
||||||
|
|
||||||
|
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_)
|
||||||
|
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -34,20 +90,7 @@ NamesAndTypesList IRowSchemaReader::readSchema()
|
|||||||
if (!new_data_types[i])
|
if (!new_data_types[i])
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
/// If we couldn't determine the type of column yet, just set the new type.
|
chooseResultType(data_types[i], new_data_types[i], allow_bools_as_numbers, default_type, std::to_string(i + 1), row);
|
||||||
if (!data_types[i])
|
|
||||||
data_types[i] = new_data_types[i];
|
|
||||||
/// If the new type and the previous type for this column are different,
|
|
||||||
/// we will use default type if we have it or throw an exception.
|
|
||||||
else if (data_types[i]->getName() != new_data_types[i]->getName())
|
|
||||||
{
|
|
||||||
if (default_type)
|
|
||||||
data_types[i] = default_type;
|
|
||||||
else
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
|
||||||
"Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -72,25 +115,14 @@ NamesAndTypesList IRowSchemaReader::readSchema()
|
|||||||
for (size_t i = 0; i != data_types.size(); ++i)
|
for (size_t i = 0; i != data_types.size(); ++i)
|
||||||
{
|
{
|
||||||
/// Check that we could determine the type of this column.
|
/// Check that we could determine the type of this column.
|
||||||
if (!data_types[i])
|
checkTypeAndAppend(result, data_types[i], column_names[i], default_type, max_rows_to_read);
|
||||||
{
|
|
||||||
if (!default_type)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
|
||||||
"Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum "
|
|
||||||
"number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference",
|
|
||||||
max_rows_to_read);
|
|
||||||
|
|
||||||
data_types[i] = default_type;
|
|
||||||
}
|
|
||||||
result.emplace_back(column_names[i], data_types[i]);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_)
|
IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_)
|
||||||
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_)
|
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -127,20 +159,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
|||||||
}
|
}
|
||||||
|
|
||||||
auto & type = it->second;
|
auto & type = it->second;
|
||||||
/// If we couldn't determine the type of column yet, just set the new type.
|
chooseResultType(type, new_type, allow_bools_as_numbers, default_type, name, row);
|
||||||
if (!type)
|
|
||||||
type = new_type;
|
|
||||||
/// If the new type and the previous type for this column are different,
|
|
||||||
/// we will use default type if we have it or throw an exception.
|
|
||||||
else if (new_type && type->getName() != new_type->getName())
|
|
||||||
{
|
|
||||||
if (default_type)
|
|
||||||
type = default_type;
|
|
||||||
else
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
|
||||||
"Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -153,18 +172,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
|
|||||||
{
|
{
|
||||||
auto & type = names_to_types[name];
|
auto & type = names_to_types[name];
|
||||||
/// Check that we could determine the type of this column.
|
/// Check that we could determine the type of this column.
|
||||||
if (!type)
|
checkTypeAndAppend(result, type, name, default_type, max_rows_to_read);
|
||||||
{
|
|
||||||
if (!default_type)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
|
||||||
"Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum "
|
|
||||||
"number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference",
|
|
||||||
max_rows_to_read);
|
|
||||||
|
|
||||||
type = default_type;
|
|
||||||
}
|
|
||||||
result.emplace_back(name, type);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
|
@ -36,7 +36,7 @@ protected:
|
|||||||
class IRowSchemaReader : public ISchemaReader
|
class IRowSchemaReader : public ISchemaReader
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
|
IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false);
|
||||||
NamesAndTypesList readSchema() override;
|
NamesAndTypesList readSchema() override;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
@ -51,6 +51,7 @@ protected:
|
|||||||
private:
|
private:
|
||||||
size_t max_rows_to_read;
|
size_t max_rows_to_read;
|
||||||
DataTypePtr default_type;
|
DataTypePtr default_type;
|
||||||
|
bool allow_bools_as_numbers;
|
||||||
std::vector<String> column_names;
|
std::vector<String> column_names;
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -62,7 +63,7 @@ private:
|
|||||||
class IRowWithNamesSchemaReader : public ISchemaReader
|
class IRowWithNamesSchemaReader : public ISchemaReader
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
|
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false);
|
||||||
NamesAndTypesList readSchema() override;
|
NamesAndTypesList readSchema() override;
|
||||||
bool hasStrictOrderOfColumns() const override { return false; }
|
bool hasStrictOrderOfColumns() const override { return false; }
|
||||||
|
|
||||||
@ -76,6 +77,7 @@ protected:
|
|||||||
private:
|
private:
|
||||||
size_t max_rows_to_read;
|
size_t max_rows_to_read;
|
||||||
DataTypePtr default_type;
|
DataTypePtr default_type;
|
||||||
|
bool allow_bools_as_numbers;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Base class for schema inference for formats that don't need any data to
|
/// Base class for schema inference for formats that don't need any data to
|
||||||
|
@ -182,7 +182,15 @@ bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer &
|
|||||||
}
|
}
|
||||||
|
|
||||||
JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_)
|
JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_)
|
||||||
: FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_)
|
: FormatWithNamesAndTypesSchemaReader(
|
||||||
|
in_,
|
||||||
|
format_settings_.max_rows_to_read_for_schema_inference,
|
||||||
|
with_names_,
|
||||||
|
with_types_,
|
||||||
|
&reader,
|
||||||
|
nullptr,
|
||||||
|
format_settings_.json.read_bools_as_numbers)
|
||||||
|
, reader(in_, yield_strings_, format_settings_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -307,7 +307,9 @@ void JSONEachRowRowInputFormat::readSuffix()
|
|||||||
}
|
}
|
||||||
|
|
||||||
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings)
|
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings)
|
||||||
: IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference), json_strings(json_strings_)
|
: IRowWithNamesSchemaReader(
|
||||||
|
in_, format_settings.max_rows_to_read_for_schema_inference, nullptr, format_settings.json.read_bools_as_numbers)
|
||||||
|
, json_strings(json_strings_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -297,8 +297,9 @@ FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader(
|
|||||||
bool with_names_,
|
bool with_names_,
|
||||||
bool with_types_,
|
bool with_types_,
|
||||||
FormatWithNamesAndTypesReader * format_reader_,
|
FormatWithNamesAndTypesReader * format_reader_,
|
||||||
DataTypePtr default_type_)
|
DataTypePtr default_type_,
|
||||||
: IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_)
|
bool allow_bools_as_numbers_)
|
||||||
|
: IRowSchemaReader(in_, max_rows_to_read_, default_type_, allow_bools_as_numbers_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -128,7 +128,8 @@ public:
|
|||||||
bool with_names_,
|
bool with_names_,
|
||||||
bool with_types_,
|
bool with_types_,
|
||||||
FormatWithNamesAndTypesReader * format_reader_,
|
FormatWithNamesAndTypesReader * format_reader_,
|
||||||
DataTypePtr default_type_ = nullptr);
|
DataTypePtr default_type_ = nullptr,
|
||||||
|
bool allow_bools_as_numbers_ = false);
|
||||||
|
|
||||||
NamesAndTypesList readSchema() override;
|
NamesAndTypesList readSchema() override;
|
||||||
|
|
||||||
|
@ -38,32 +38,32 @@ JSONCompactEachRow
|
|||||||
c1 Nullable(Float64)
|
c1 Nullable(Float64)
|
||||||
c2 Array(Tuple(Nullable(Float64), Nullable(String)))
|
c2 Array(Tuple(Nullable(Float64), Nullable(String)))
|
||||||
c3 Map(String, Nullable(Float64))
|
c3 Map(String, Nullable(Float64))
|
||||||
c4 Nullable(UInt8)
|
c4 Nullable(Bool)
|
||||||
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1
|
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true
|
||||||
c1 Nullable(Float64)
|
c1 Nullable(Float64)
|
||||||
c2 Array(Tuple(Nullable(Float64), Nullable(String)))
|
c2 Array(Tuple(Nullable(Float64), Nullable(String)))
|
||||||
c3 Map(String, Nullable(Float64))
|
c3 Map(String, Nullable(Float64))
|
||||||
c4 Nullable(UInt8)
|
c4 Nullable(Bool)
|
||||||
\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N
|
\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N
|
||||||
32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1
|
32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true
|
||||||
JSONCompactEachRowWithNames
|
JSONCompactEachRowWithNames
|
||||||
a Nullable(Float64)
|
a Nullable(Float64)
|
||||||
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
||||||
c Map(String, Nullable(Float64))
|
c Map(String, Nullable(Float64))
|
||||||
d Nullable(UInt8)
|
d Nullable(Bool)
|
||||||
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1
|
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true
|
||||||
JSONEachRow
|
JSONEachRow
|
||||||
a Nullable(Float64)
|
a Nullable(Float64)
|
||||||
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
||||||
c Map(String, Nullable(Float64))
|
c Map(String, Nullable(Float64))
|
||||||
d Nullable(UInt8)
|
d Nullable(Bool)
|
||||||
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1
|
42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true
|
||||||
a Nullable(Float64)
|
a Nullable(Float64)
|
||||||
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
b Array(Tuple(Nullable(Float64), Nullable(String)))
|
||||||
c Map(String, Nullable(Float64))
|
c Map(String, Nullable(Float64))
|
||||||
d Nullable(UInt8)
|
d Nullable(Bool)
|
||||||
\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N
|
\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N
|
||||||
32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1
|
32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true
|
||||||
a Nullable(Float64)
|
a Nullable(Float64)
|
||||||
b Nullable(String)
|
b Nullable(String)
|
||||||
c Array(Nullable(Float64))
|
c Array(Nullable(Float64))
|
||||||
|
@ -0,0 +1,18 @@
|
|||||||
|
x Nullable(Bool)
|
||||||
|
true
|
||||||
|
false
|
||||||
|
x Nullable(Float64)
|
||||||
|
42.42
|
||||||
|
0
|
||||||
|
x Nullable(Float64)
|
||||||
|
1
|
||||||
|
0.42
|
||||||
|
c1 Nullable(Bool)
|
||||||
|
true
|
||||||
|
false
|
||||||
|
c1 Nullable(Float64)
|
||||||
|
42.42
|
||||||
|
0
|
||||||
|
c1 Nullable(Float64)
|
||||||
|
1
|
||||||
|
0.42
|
47
tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh
Executable file
47
tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh
Executable file
@ -0,0 +1,47 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
# Tags: no-parallel, no-fasttest
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CURDIR"/../shell_config.sh
|
||||||
|
|
||||||
|
|
||||||
|
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||||
|
FILE_NAME=test_02247.data
|
||||||
|
DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME
|
||||||
|
|
||||||
|
touch $DATA_FILE
|
||||||
|
|
||||||
|
echo -e '{"x" : true}
|
||||||
|
{"x" : false}' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
|
||||||
|
echo -e '{"x" : 42.42}
|
||||||
|
{"x" : false}' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
|
||||||
|
echo -e '{"x" : true}
|
||||||
|
{"x" : 0.42}' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')"
|
||||||
|
|
||||||
|
|
||||||
|
echo -e '[true]
|
||||||
|
[false]' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
|
||||||
|
echo -e '[42.42]
|
||||||
|
[false]' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
|
||||||
|
echo -e '[true]
|
||||||
|
[0.42]' > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')"
|
||||||
|
|
||||||
|
|
||||||
|
rm $DATA_FILE
|
Loading…
Reference in New Issue
Block a user