Merge branch 'master' into add_one_more_group

This commit is contained in:
mergify[bot] 2021-12-02 12:56:57 +00:00 committed by GitHub
commit 88e1a26e99
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
42 changed files with 567 additions and 83 deletions

View File

@ -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) \

View 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);
}
}

View File

@ -209,6 +209,7 @@ DataTypeFactory::DataTypeFactory()
registerDataTypeInterval(*this);
registerDataTypeLowCardinality(*this);
registerDataTypeDomainIPv4AndIPv6(*this);
registerDataTypeDomainBool(*this);
registerDataTypeDomainSimpleAggregateFunction(*this);
registerDataTypeDomainGeo(*this);
registerDataTypeMap(*this);

View File

@ -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);

View File

@ -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);

View 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);
}
}

View 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);
};
}

View File

@ -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;

View File

@ -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 = ',';

View File

@ -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;
};

View File

@ -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);

View File

@ -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)

View File

@ -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)

View File

@ -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);

View File

@ -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);

View File

@ -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)

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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();

View File

@ -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)

View File

@ -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;

View File

@ -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.
*/

View File

@ -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

View File

@ -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; }

View File

@ -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))

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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")

View File

@ -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):

View File

@ -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")

View File

@ -16,7 +16,7 @@ c
a
1
1
1
true
1
a
\N

View 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

View 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;

View File

@ -0,0 +1 @@
Value

View File

@ -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}

View File

@ -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

View 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"