mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge branch 'master' into add_one_more_group
This commit is contained in:
commit
88e1a26e99
@ -592,6 +592,9 @@ class IColumn;
|
||||
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 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(String, bool_true_representation, "true", "Text to represent bool value in TSV/CSV formats.", 0) \
|
||||
M(String, bool_false_representation, "false", "Text to represent bool value in TSV/CSV formats.", 0) \
|
||||
\
|
||||
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
|
||||
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
|
||||
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
|
||||
|
21
src/DataTypes/DataTypeDomainBool.cpp
Normal file
21
src/DataTypes/DataTypeDomainBool.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <DataTypes/Serializations/SerializationBool.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDataTypeDomainBool(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataTypeCustom("Bool", []
|
||||
{
|
||||
auto type = DataTypeFactory::instance().get("UInt8");
|
||||
return std::make_pair(type, std::make_unique<DataTypeCustomDesc>(
|
||||
std::make_unique<DataTypeCustomFixedName>("Bool"), std::make_unique<SerializationBool>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -209,6 +209,7 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeInterval(*this);
|
||||
registerDataTypeLowCardinality(*this);
|
||||
registerDataTypeDomainIPv4AndIPv6(*this);
|
||||
registerDataTypeDomainBool(*this);
|
||||
registerDataTypeDomainSimpleAggregateFunction(*this);
|
||||
registerDataTypeDomainGeo(*this);
|
||||
registerDataTypeMap(*this);
|
||||
|
@ -85,6 +85,7 @@ void registerDataTypeNested(DataTypeFactory & factory);
|
||||
void registerDataTypeInterval(DataTypeFactory & factory);
|
||||
void registerDataTypeLowCardinality(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainBool(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainGeo(DataTypeFactory & factory);
|
||||
|
||||
|
@ -57,8 +57,6 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
|
||||
/// These synonyms are added for compatibility.
|
||||
|
||||
factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BOOL", "Int8", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BOOLEAN", "Int8", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); /// MySQL
|
||||
factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); /// MS Access
|
||||
factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive);
|
||||
|
169
src/DataTypes/Serializations/SerializationBool.cpp
Normal file
169
src/DataTypes/Serializations/SerializationBool.cpp
Normal file
@ -0,0 +1,169 @@
|
||||
#include <DataTypes/Serializations/SerializationBool.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
SerializationBool::SerializationBool(const SerializationPtr &nested_)
|
||||
: SerializationCustomSimpleText(nested_)
|
||||
{
|
||||
}
|
||||
|
||||
void SerializationBool::serializeText(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &) const
|
||||
{
|
||||
const auto *col = checkAndGetColumn<ColumnUInt8>(&column);
|
||||
if (!col)
|
||||
throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (col->getData()[row_num])
|
||||
ostr.write(str_true, sizeof(str_true) - 1);
|
||||
else
|
||||
ostr.write(str_false, sizeof(str_false) - 1);
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeText(IColumn &column, ReadBuffer &istr, const FormatSettings & settings, bool whole) const
|
||||
{
|
||||
ColumnUInt8 *col = typeid_cast<ColumnUInt8 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (!istr.eof())
|
||||
{
|
||||
bool value = false;
|
||||
|
||||
if (*istr.position() == 't' || *istr.position() == 'f' || *istr.position() == 'T' || *istr.position() == 'F')
|
||||
readBoolTextWord(value, istr, true);
|
||||
else if (*istr.position() == '1' || *istr.position() == '0')
|
||||
readBoolText(value, istr);
|
||||
else
|
||||
throw Exception("Invalid boolean value, should be true/false, TRUE/FALSE, 1/0.",
|
||||
ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
col->insert(value);
|
||||
}
|
||||
else
|
||||
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
|
||||
if (whole && !istr.eof())
|
||||
throwUnexpectedDataAfterParsedValue(column, istr, settings, "Bool");
|
||||
}
|
||||
|
||||
void SerializationBool::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
const auto *col = checkAndGetColumn<ColumnUInt8>(&column);
|
||||
if (!col)
|
||||
throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (col->getData()[row_num])
|
||||
{
|
||||
writeString(settings.bool_true_representation, ostr);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeString(settings.bool_false_representation, ostr);
|
||||
}
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (istr.eof())
|
||||
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
String input;
|
||||
readEscapedString(input, istr);
|
||||
deserializeFromString(column, input, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, const FormatSettings &) const
|
||||
{
|
||||
ColumnUInt8 *col = typeid_cast<ColumnUInt8 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (!istr.eof())
|
||||
{
|
||||
bool value = false;
|
||||
|
||||
if (*istr.position() == 't' || *istr.position() == 'f')
|
||||
readBoolTextWord(value, istr);
|
||||
else if (*istr.position() == '1' || *istr.position() == '0')
|
||||
readBoolText(value, istr);
|
||||
else
|
||||
throw Exception("Invalid boolean value, should be true/false, 1/0.",
|
||||
ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
col->insert(value);
|
||||
}
|
||||
else
|
||||
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeTextEscaped(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (istr.eof())
|
||||
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
String input;
|
||||
readCSVString(input, istr, settings.csv);
|
||||
deserializeFromString(column, input, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeTextEscaped(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (istr.eof())
|
||||
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
String input;
|
||||
readString(input, istr);
|
||||
deserializeFromString(column, input, settings);
|
||||
}
|
||||
|
||||
void SerializationBool::deserializeFromString(IColumn & column, String & input, const FormatSettings & settings)
|
||||
{
|
||||
ColumnUInt8 * col = typeid_cast<ColumnUInt8 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (settings.bool_true_representation == input)
|
||||
{
|
||||
col->insert(true);
|
||||
}
|
||||
else if (settings.bool_false_representation == input)
|
||||
{
|
||||
col->insert(false);
|
||||
}
|
||||
else
|
||||
throw Exception("Invalid boolean value, should be " + settings.bool_true_representation + " or " + settings.bool_false_representation + " controlled by setting bool_true_representation and bool_false_representation.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
36
src/DataTypes/Serializations/SerializationBool.h
Normal file
36
src/DataTypes/Serializations/SerializationBool.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationCustomSimpleText.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationBool final : public SerializationCustomSimpleText
|
||||
{
|
||||
private:
|
||||
static constexpr char str_true[5] = "true";
|
||||
static constexpr char str_false[6] = "false";
|
||||
|
||||
public:
|
||||
SerializationBool(const SerializationPtr & nested_);
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings,bool whole) const override;
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
protected:
|
||||
static void deserializeFromString(IColumn & column, String & input, const FormatSettings & settings);
|
||||
};
|
||||
|
||||
}
|
@ -70,6 +70,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
|
||||
format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter;
|
||||
format_settings.date_time_input_format = settings.date_time_input_format;
|
||||
format_settings.date_time_output_format = settings.date_time_output_format;
|
||||
format_settings.bool_true_representation = settings.bool_true_representation;
|
||||
format_settings.bool_false_representation = settings.bool_false_representation;
|
||||
format_settings.enable_streaming = settings.output_format_enable_streaming;
|
||||
format_settings.import_nested_json = settings.input_format_import_nested_json;
|
||||
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
|
||||
|
@ -82,6 +82,9 @@ struct FormatSettings
|
||||
UInt64 output_rows_in_file = 1;
|
||||
} avro;
|
||||
|
||||
String bool_true_representation = "true";
|
||||
String bool_false_representation = "false";
|
||||
|
||||
struct CSV
|
||||
{
|
||||
char delimiter = ',';
|
||||
|
@ -662,18 +662,13 @@ private:
|
||||
{
|
||||
auto return_type = impl.getReturnTypeImpl(arguments);
|
||||
|
||||
if (!areTypesEqual(return_type, result_type))
|
||||
if (!return_type->equals(*result_type))
|
||||
throw Exception{"Dictionary attribute has different type " + return_type->getName() + " expected " + result_type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
|
||||
return impl.executeImpl(arguments, return_type, input_rows_count);
|
||||
}
|
||||
|
||||
static bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs)
|
||||
{
|
||||
return removeNullable(recursiveRemoveLowCardinality(lhs))->equals(*removeNullable(recursiveRemoveLowCardinality(rhs)));
|
||||
}
|
||||
|
||||
const FunctionDictGetNoType<dictionary_get_function_type> impl;
|
||||
};
|
||||
|
||||
|
@ -76,7 +76,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const double lon = col_lon->getFloat64(row);
|
||||
const double lat = col_lat->getFloat64(row);
|
||||
|
@ -58,7 +58,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const int resolution = col_hindex->getUInt(row);
|
||||
if (resolution > MAX_H3_RES)
|
||||
|
@ -63,7 +63,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 resolution = col_hindex->getUInt(row);
|
||||
if (resolution > MAX_H3_RES)
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
|
||||
|
@ -58,7 +58,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 resolution = col_hindex->getUInt(row);
|
||||
if (resolution > MAX_H3_RES)
|
||||
|
@ -63,7 +63,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 hindex_origin = col_hindex_origin->getUInt(row);
|
||||
const UInt64 hindex_dest = col_hindex_dest->getUInt(row);
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
|
||||
|
@ -76,7 +76,7 @@ public:
|
||||
|
||||
std::vector<H3Index> hindex_vec;
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 parent_hindex = col_hindex->getUInt(row);
|
||||
const UInt8 child_resolution = col_resolution->getUInt(row);
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
const UInt8 resolution = col_resolution->getUInt(row);
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
|
||||
std::vector<H3Index> hindex_vec;
|
||||
|
||||
for (const auto row : collections::range(0, input_rows_count))
|
||||
for (size_t row = 0; row < input_rows_count; row++)
|
||||
{
|
||||
const H3Index origin_hindex = col_hindex->getUInt(row);
|
||||
const int k = col_k->getInt(row);
|
||||
|
@ -48,6 +48,7 @@ struct Memory;
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_DATE;
|
||||
extern const int CANNOT_PARSE_BOOL;
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_UUID;
|
||||
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
|
||||
@ -231,20 +232,45 @@ inline void readBoolText(bool & x, ReadBuffer & buf)
|
||||
x = tmp != '0';
|
||||
}
|
||||
|
||||
inline void readBoolTextWord(bool & x, ReadBuffer & buf)
|
||||
inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case = false)
|
||||
{
|
||||
if (buf.eof())
|
||||
throwReadAfterEOF();
|
||||
|
||||
if (*buf.position() == 't')
|
||||
switch (*buf.position())
|
||||
{
|
||||
assertString("true", buf);
|
||||
x = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
assertString("false", buf);
|
||||
x = false;
|
||||
case 't':
|
||||
assertString("true", buf);
|
||||
x = true;
|
||||
break;
|
||||
case 'f':
|
||||
assertString("false", buf);
|
||||
x = false;
|
||||
break;
|
||||
case 'T':
|
||||
{
|
||||
if (support_upper_case)
|
||||
{
|
||||
assertString("TRUE", buf);
|
||||
x = true;
|
||||
break;
|
||||
}
|
||||
else
|
||||
[[fallthrough]];
|
||||
}
|
||||
case 'F':
|
||||
{
|
||||
if (support_upper_case)
|
||||
{
|
||||
assertString("FALSE", buf);
|
||||
x = false;
|
||||
break;
|
||||
}
|
||||
else
|
||||
[[fallthrough]];
|
||||
}
|
||||
default:
|
||||
throw ParsingException("Unexpected Bool value", ErrorCodes::CANNOT_PARSE_BOOL);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -341,6 +341,8 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
|
||||
negative = true;
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '+')
|
||||
++in.position();
|
||||
|
||||
auto count_after_sign = in.count();
|
||||
|
||||
|
@ -40,7 +40,7 @@ TEST(MySQLCreateRewritten, ColumnsDataType)
|
||||
{"TINYINT", "Int8"}, {"SMALLINT", "Int16"}, {"MEDIUMINT", "Int32"}, {"INT", "Int32"},
|
||||
{"INTEGER", "Int32"}, {"BIGINT", "Int64"}, {"FLOAT", "Float32"}, {"DOUBLE", "Float64"},
|
||||
{"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date"}, {"DateTime", "DateTime"},
|
||||
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Int8"}
|
||||
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Bool"}
|
||||
};
|
||||
|
||||
for (const auto & [test_type, mapped_type] : test_types)
|
||||
@ -104,7 +104,7 @@ TEST(MySQLCreateRewritten, PartitionPolicy)
|
||||
{"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"},
|
||||
{"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""},
|
||||
{"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
|
||||
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"}
|
||||
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"}
|
||||
};
|
||||
|
||||
for (const auto & [test_type, mapped_type, partition_policy] : test_types)
|
||||
@ -135,7 +135,7 @@ TEST(MySQLCreateRewritten, OrderbyPolicy)
|
||||
{"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"},
|
||||
{"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""},
|
||||
{"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
|
||||
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"}
|
||||
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"}
|
||||
};
|
||||
|
||||
for (const auto & [test_type, mapped_type, partition_policy] : test_types)
|
||||
|
@ -1523,6 +1523,23 @@ bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
bool ParserBool::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (ParserKeyword("true").parse(pos, node, expected))
|
||||
{
|
||||
node = std::make_shared<ASTLiteral>(true);
|
||||
return true;
|
||||
}
|
||||
else if (ParserKeyword("false").parse(pos, node, expected))
|
||||
{
|
||||
node = std::make_shared<ASTLiteral>(false);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
static bool parseNumber(char * buffer, size_t size, bool negative, int base, Field & res)
|
||||
{
|
||||
errno = 0; /// Functions strto* don't clear errno.
|
||||
@ -1754,6 +1771,7 @@ bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserNull null_p;
|
||||
ParserNumber num_p;
|
||||
ParserBool bool_p;
|
||||
ParserStringLiteral str_p;
|
||||
|
||||
if (null_p.parse(pos, node, expected))
|
||||
@ -1762,6 +1780,9 @@ bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (num_p.parse(pos, node, expected))
|
||||
return true;
|
||||
|
||||
if (bool_p.parse(pos, node, expected))
|
||||
return true;
|
||||
|
||||
if (str_p.parse(pos, node, expected))
|
||||
return true;
|
||||
|
||||
|
@ -294,6 +294,14 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/** Bool literal.
|
||||
*/
|
||||
class ParserBool : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "Bool"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/** Numeric literal.
|
||||
*/
|
||||
|
@ -1283,7 +1283,7 @@ void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_sh
|
||||
}
|
||||
}
|
||||
|
||||
String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const
|
||||
String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached) const
|
||||
{
|
||||
String res;
|
||||
|
||||
@ -1292,11 +1292,20 @@ String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const
|
||||
* This is done only in the case of `to_detached`, because it is assumed that in this case the exact name does not matter.
|
||||
* No more than 10 attempts are made so that there are not too many junk directories left.
|
||||
*/
|
||||
|
||||
auto full_relative_path = fs::path(storage.relative_data_path);
|
||||
if (detached)
|
||||
full_relative_path /= "detached";
|
||||
if (detached && parent_part)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection");
|
||||
else if (parent_part)
|
||||
full_relative_path /= parent_part->relative_path;
|
||||
|
||||
for (int try_no = 0; try_no < 10; try_no++)
|
||||
{
|
||||
res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : "");
|
||||
|
||||
if (!volume->getDisk()->exists(fs::path(getFullRelativePath()) / res))
|
||||
if (!volume->getDisk()->exists(full_relative_path / res))
|
||||
return res;
|
||||
|
||||
LOG_WARNING(storage.log, "Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix.", res);
|
||||
@ -1312,7 +1321,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix)
|
||||
assert(prefix.empty() || std::find(DetachedPartInfo::DETACH_REASONS.begin(),
|
||||
DetachedPartInfo::DETACH_REASONS.end(),
|
||||
prefix) != DetachedPartInfo::DETACH_REASONS.end());
|
||||
return "detached/" + getRelativePathForPrefix(prefix);
|
||||
return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true);
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::renameToDetached(const String & prefix) const
|
||||
|
@ -357,7 +357,7 @@ public:
|
||||
/// Calculate column and secondary indices sizes on disk.
|
||||
void calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
||||
|
||||
String getRelativePathForPrefix(const String & prefix) const;
|
||||
String getRelativePathForPrefix(const String & prefix, bool detached = false) const;
|
||||
|
||||
bool isProjectionPart() const { return parent_part != nullptr; }
|
||||
|
||||
|
@ -3339,7 +3339,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
|
||||
/// It's quite dangerous, so clone covered parts to detached.
|
||||
auto broken_part_info = MergeTreePartInfo::fromPartName(part_name, format_version);
|
||||
|
||||
auto partition_range = getDataPartsPartitionRange(broken_part_info.partition_id);
|
||||
auto partition_range = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, broken_part_info.partition_id);
|
||||
for (const auto & part : partition_range)
|
||||
{
|
||||
if (!broken_part_info.contains(part->info))
|
||||
|
@ -21,18 +21,4 @@
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
||||
|
@ -21,18 +21,4 @@
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
||||
|
@ -20,18 +20,4 @@
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>node1</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>node2</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>node3</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
</yandex>
|
||||
|
@ -3,7 +3,7 @@ import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
import time
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster = ClickHouseCluster(__file__, name="test_keeper_4lw_white_list")
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config_with_white_list.xml'], stay_alive=True)
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/keeper_config_without_white_list.xml'], stay_alive=True)
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/keeper_config_with_white_list_all.xml'], stay_alive=True)
|
||||
@ -35,7 +35,6 @@ def wait_node(node):
|
||||
for _ in range(100):
|
||||
zk = None
|
||||
try:
|
||||
node.query("SELECT * FROM system.zookeeper WHERE path = '/'")
|
||||
zk = get_fake_zk(node.name, timeout=30.0)
|
||||
# zk.create("/test", sequence=True)
|
||||
print("node", node.name, "ready")
|
||||
|
@ -36,6 +36,16 @@ def start(node):
|
||||
node.start_clickhouse()
|
||||
|
||||
|
||||
def delete_with_retry(node_name, path):
|
||||
for _ in range(30):
|
||||
try:
|
||||
get_fake_zk(node_name).delete(path)
|
||||
return
|
||||
except:
|
||||
time.sleep(0.5)
|
||||
raise Exception(f"Cannot delete {path} from node {node_name}")
|
||||
|
||||
|
||||
def test_start_offline(started_cluster):
|
||||
p = Pool(3)
|
||||
try:
|
||||
@ -57,7 +67,7 @@ def test_start_offline(started_cluster):
|
||||
|
||||
finally:
|
||||
p.map(start, [node1, node2, node3])
|
||||
get_fake_zk("node1").delete("/test_alive")
|
||||
delete_with_retry("node1", "/test_alive")
|
||||
|
||||
|
||||
def test_start_non_existing(started_cluster):
|
||||
@ -82,7 +92,7 @@ def test_start_non_existing(started_cluster):
|
||||
node1.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper1.xml', 'non_existing_node', 'node3')
|
||||
node2.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper2.xml', 'non_existing_node', 'node3')
|
||||
p.map(start, [node1, node2, node3])
|
||||
node2_zk.delete("/test_non_exising")
|
||||
delete_with_retry("node2", "/test_non_exising")
|
||||
|
||||
|
||||
def test_restart_third_node(started_cluster):
|
||||
|
@ -315,3 +315,19 @@ def test_system_detached_parts(drop_detached_parts_table):
|
||||
q("alter table sdp_{} attach partition id '{}'".format(i, p))
|
||||
|
||||
assert q("select n, x, count() from merge('default', 'sdp_') group by n, x") == "0\t0\t4\n1\t1\t4\n"
|
||||
|
||||
|
||||
def test_detached_part_dir_exists(started_cluster):
|
||||
q("create table detached_part_dir_exists (n int) engine=MergeTree order by n")
|
||||
q("insert into detached_part_dir_exists select 1") # will create all_1_1_0
|
||||
q("alter table detached_part_dir_exists detach partition id 'all'") # will move all_1_1_0 to detached/all_1_1_0
|
||||
q("detach table detached_part_dir_exists")
|
||||
q("attach table detached_part_dir_exists")
|
||||
q("insert into detached_part_dir_exists select 1") # will create all_1_1_0
|
||||
q("insert into detached_part_dir_exists select 1") # will create all_2_2_0
|
||||
instance.exec_in_container(['bash', '-c', 'mkdir /var/lib/clickhouse/data/default/detached_part_dir_exists/detached/all_2_2_0'], privileged=True)
|
||||
instance.exec_in_container(['bash', '-c', 'touch /var/lib/clickhouse/data/default/detached_part_dir_exists/detached/all_2_2_0/file'], privileged=True)
|
||||
q("alter table detached_part_dir_exists detach partition id 'all'") # directories already exist, but it's ok
|
||||
assert q("select name from system.detached_parts where table='detached_part_dir_exists' order by name") == \
|
||||
"all_1_1_0\nall_1_1_0_try1\nall_2_2_0\nall_2_2_0_try1\n"
|
||||
q("drop table detached_part_dir_exists")
|
||||
|
@ -16,7 +16,7 @@ c
|
||||
a
|
||||
1
|
||||
1
|
||||
1
|
||||
true
|
||||
1
|
||||
a
|
||||
\N
|
||||
|
120
tests/queries/0_stateless/02114_bool_type.reference
Normal file
120
tests/queries/0_stateless/02114_bool_type.reference
Normal file
@ -0,0 +1,120 @@
|
||||
CREATE TABLE default.bool_test\n(\n `value` Bool,\n `f` String\n)\nENGINE = Memory
|
||||
false test
|
||||
true test
|
||||
false test
|
||||
true test
|
||||
false test
|
||||
true test
|
||||
false test
|
||||
true test
|
||||
false test
|
||||
true test
|
||||
{"value":false,"f":"test"}
|
||||
{"value":true,"f":"test"}
|
||||
{"value":false,"f":"test"}
|
||||
{"value":true,"f":"test"}
|
||||
{"value":false,"f":"test"}
|
||||
{"value":true,"f":"test"}
|
||||
{"value":false,"f":"test"}
|
||||
{"value":true,"f":"test"}
|
||||
{"value":false,"f":"test"}
|
||||
{"value":true,"f":"test"}
|
||||
0 test
|
||||
1 test
|
||||
0 test
|
||||
1 test
|
||||
0 test
|
||||
1 test
|
||||
0 test
|
||||
1 test
|
||||
0 test
|
||||
1 test
|
||||
true test
|
||||
true test
|
||||
true test
|
||||
true test
|
||||
true test
|
||||
False,"test"
|
||||
False,"test"
|
||||
False,"test"
|
||||
False,"test"
|
||||
False,"test"
|
||||
False,"test"
|
||||
True,"test"
|
||||
True,"test"
|
||||
True,"test"
|
||||
True,"test"
|
||||
True,"test"
|
||||
True,"test"
|
||||
False test
|
||||
False test
|
||||
False test
|
||||
False test
|
||||
False test
|
||||
False test
|
||||
True test
|
||||
True test
|
||||
True test
|
||||
True test
|
||||
True test
|
||||
True test
|
||||
No,"test"
|
||||
No,"test"
|
||||
No,"test"
|
||||
No,"test"
|
||||
No,"test"
|
||||
No,"test"
|
||||
No,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
Yes,"test"
|
||||
No test
|
||||
No test
|
||||
No test
|
||||
No test
|
||||
No test
|
||||
No test
|
||||
No test
|
||||
Yes test
|
||||
Yes test
|
||||
Yes test
|
||||
Yes test
|
||||
Yes test
|
||||
Yes test
|
||||
Yes test
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
Off,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
On,"test"
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
Off test
|
||||
On test
|
||||
On test
|
||||
On test
|
||||
On test
|
||||
On test
|
||||
On test
|
||||
On test
|
||||
On test
|
38
tests/queries/0_stateless/02114_bool_type.sql
Normal file
38
tests/queries/0_stateless/02114_bool_type.sql
Normal file
@ -0,0 +1,38 @@
|
||||
DROP TABLE IF EXISTS bool_test;
|
||||
|
||||
CREATE TABLE bool_test (value Bool,f String) ENGINE = Memory;
|
||||
|
||||
-- value column shoud have type 'Bool'
|
||||
SHOW CREATE TABLE bool_test;
|
||||
|
||||
INSERT INTO bool_test (value,f) VALUES ('false', 'test'), ('true' , 'test'), (0, 'test'), (1, 'test'), ('FALSE', 'test'), ('TRUE', 'test');
|
||||
INSERT INTO bool_test (value,f) FORMAT JSONEachRow {"value":false,"f":"test"}{"value":true,"f":"test"}{"value":0,"f":"test"}{"value":1,"f":"test"}
|
||||
|
||||
SELECT value,f FROM bool_test;
|
||||
SELECT value,f FROM bool_test FORMAT JSONEachRow;
|
||||
SELECT toUInt64(value),f FROM bool_test;
|
||||
SELECT value,f FROM bool_test where value > 0;
|
||||
|
||||
set bool_true_representation='True';
|
||||
set bool_false_representation='False';
|
||||
INSERT INTO bool_test (value,f) FORMAT CSV True,test
|
||||
INSERT INTO bool_test (value,f) FORMAT TSV False test
|
||||
SELECT value,f FROM bool_test order by value FORMAT CSV;
|
||||
SELECT value,f FROM bool_test order by value FORMAT TSV;
|
||||
|
||||
set bool_true_representation='Yes';
|
||||
set bool_false_representation='No';
|
||||
INSERT INTO bool_test (value,f) FORMAT CSV Yes,test
|
||||
INSERT INTO bool_test (value,f) FORMAT TSV No test
|
||||
SELECT value,f FROM bool_test order by value FORMAT CSV;
|
||||
SELECT value,f FROM bool_test order by value FORMAT TSV;
|
||||
|
||||
set bool_true_representation='On';
|
||||
set bool_false_representation='Off';
|
||||
INSERT INTO bool_test (value,f) FORMAT CSV On,test
|
||||
INSERT INTO bool_test (value,f) FORMAT TSV Off test
|
||||
SELECT value,f FROM bool_test order by value FORMAT CSV;
|
||||
SELECT value,f FROM bool_test order by value FORMAT TSV;
|
||||
|
||||
DROP TABLE IF EXISTS bool_test;
|
||||
|
@ -0,0 +1 @@
|
||||
Value
|
@ -0,0 +1,22 @@
|
||||
DROP TABLE IF EXISTS 02125_test_table;
|
||||
CREATE TABLE 02125_test_table
|
||||
(
|
||||
id UInt64,
|
||||
value Nullable(String)
|
||||
)
|
||||
ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02125_test_table VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02125_test_dictionary;
|
||||
CREATE DICTIONARY 02125_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02125_test_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT dictGet('02125_test_dictionary', 'value', toUInt64(0));
|
||||
SELECT dictGetString('02125_test_dictionary', 'value', toUInt64(0)); --{serverError 53}
|
@ -0,0 +1,6 @@
|
||||
42.42 42.42
|
||||
42.42 42.42
|
||||
42.42 42.42
|
||||
42.42 42.42
|
||||
42.42 42.42
|
||||
42.42 42.42
|
19
tests/queries/0_stateless/02127_plus_before_float.sh
Executable file
19
tests/queries/0_stateless/02127_plus_before_float.sh
Executable file
@ -0,0 +1,19 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select toFloat32('+42.42'), toFloat64('+42.42')"
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists test_02127"
|
||||
$CLICKHOUSE_CLIENT -q "create table test_02127 (x Float32, y Float64) engine=Memory()"
|
||||
|
||||
for escaping_rule in Quoted JSON Escaped CSV Raw
|
||||
do
|
||||
echo -e "+42.42\t+42.42" | $CLICKHOUSE_CLIENT -q "insert into test_02127 format CustomSeparated settings format_custom_escaping_rule='$escaping_rule'"
|
||||
done
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select * from test_02127"
|
||||
$CLICKHOUSE_CLIENT -q "drop table test_02127"
|
Loading…
Reference in New Issue
Block a user