Merge branch 'master' into improve_replica_recovery

This commit is contained in:
Alexander Tokmakov 2022-10-20 15:52:48 +03:00 committed by GitHub
commit f814a17fa2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 63 additions and 6 deletions

View File

@ -11,6 +11,7 @@
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/getLeastSupertype.h>
@ -875,4 +876,19 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo
return result;
}
void checkSupportedDelimiterAfterField(FormatSettings::EscapingRule escaping_rule, const String & delimiter, const DataTypePtr & type)
{
if (escaping_rule != FormatSettings::EscapingRule::Escaped)
return;
bool is_supported_delimiter_after_string = !delimiter.empty() && (delimiter.front() == '\t' || delimiter.front() == '\n');
if (is_supported_delimiter_after_string)
return;
/// Nullptr means that field is skipped and it's equivalent to String
if (!type || isString(removeNullable(removeLowCardinality(type))))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "'Escaped' serialization requires delimiter after String field to start with '\\t' or '\\n'");
}
}

View File

@ -77,6 +77,8 @@ void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, c
void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, const std::unordered_set<const IDataType *> * numbers_parsed_from_json_strings = nullptr);
void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings);
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings,FormatSettings::EscapingRule escaping_rule);
String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, FormatSettings::EscapingRule escaping_rule);
void checkSupportedDelimiterAfterField(FormatSettings::EscapingRule escaping_rule, const String & delimiter, const DataTypePtr & type);
}

View File

@ -67,6 +67,19 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
}
}
void CustomSeparatedRowInputFormat::readPrefix()
{
RowInputFormatWithNamesAndTypes::readPrefix();
/// Provide better error message for unsupported delimiters
for (const auto & column_index : column_mapping->column_indexes_for_input_fields)
{
if (column_index)
checkSupportedDelimiterAfterField(format_settings.custom.escaping_rule, format_settings.custom.field_delimiter, data_types[*column_index]);
else
checkSupportedDelimiterAfterField(format_settings.custom.escaping_rule, format_settings.custom.field_delimiter, nullptr);
}
}
bool CustomSeparatedRowInputFormat::allowSyncAfterError() const
{

View File

@ -30,6 +30,7 @@ private:
bool allowSyncAfterError() const override;
void syncAfterError() override;
void readPrefix() override;
std::unique_ptr<PeekableReadBuffer> buf;
bool ignore_spaces;

View File

@ -53,18 +53,25 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::uniqu
std::vector<UInt8> column_in_format(header_.columns(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.format_idx_to_column_idx[i])
const auto & column_index = row_format.format_idx_to_column_idx[i];
if (column_index)
{
if (header_.columns() <= *row_format.format_idx_to_column_idx[i])
row_format.throwInvalidFormat("Column index " + std::to_string(*row_format.format_idx_to_column_idx[i]) +
if (header_.columns() <= *column_index)
row_format.throwInvalidFormat("Column index " + std::to_string(*column_index) +
" must be less then number of columns (" + std::to_string(header_.columns()) + ")", i);
if (row_format.escaping_rules[i] == EscapingRule::None)
row_format.throwInvalidFormat("Column is not skipped, but deserialization type is None", i);
size_t col_idx = *row_format.format_idx_to_column_idx[i];
size_t col_idx = *column_index;
if (column_in_format[col_idx])
row_format.throwInvalidFormat("Duplicate column", i);
column_in_format[col_idx] = true;
checkSupportedDelimiterAfterField(row_format.escaping_rules[i], row_format.delimiters[i + 1], data_types[*column_index]);
}
else
{
checkSupportedDelimiterAfterField(row_format.escaping_rules[i], row_format.delimiters[i + 1], nullptr);
}
}

View File

@ -41,6 +41,7 @@ protected:
void resetParser() override;
bool isGarbageAfterField(size_t index, ReadBuffer::Position pos) override;
void setReadBuffer(ReadBuffer & in_) override;
void readPrefix() override;
const FormatSettings format_settings;
DataTypes data_types;
@ -48,7 +49,6 @@ protected:
private:
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;

View File

@ -31,3 +31,5 @@ cv bn m","qwe,rty",456,"2016-01-02"
"zx\cv\bn m","qwe,rty","as""df'gh","",789,"2016-01-04"
"","zx
cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03"
1
1

View File

@ -83,3 +83,13 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE template1";
$CLICKHOUSE_CLIENT --query="DROP TABLE template2";
rm "$CURDIR"/00938_template_input_format_resultset.tmp "$CURDIR"/00938_template_input_format_row.tmp
echo -ne '\${a:Escaped},\${b:Escaped}\n' > "$CURDIR"/00938_template_input_format_row.tmp
echo -ne "a,b\nc,d\n" | $CLICKHOUSE_LOCAL --structure "a String, b String" --input-format Template \
--format_template_row "$CURDIR"/00938_template_input_format_row.tmp --format_template_rows_between_delimiter '' \
-q 'select * from table' 2>&1| grep -Fac "'Escaped' serialization requires delimiter"
echo -ne '\${a:Escaped},\${:Escaped}\n' > "$CURDIR"/00938_template_input_format_row.tmp
echo -ne "a,b\nc,d\n" | $CLICKHOUSE_LOCAL --structure "a String" --input-format Template \
--format_template_row "$CURDIR"/00938_template_input_format_row.tmp --format_template_rows_between_delimiter '' \
-q 'select * from table' 2>&1| grep -Fac "'Escaped' serialization requires delimiter"
rm "$CURDIR"/00938_template_input_format_row.tmp

View File

@ -8,3 +8,4 @@
1,"2019-09-25","world"
2,"2019-09-26","custom"
3,"2019-09-27","separated"
1

View File

@ -34,3 +34,8 @@ FORMAT CustomSeparated"
$CLICKHOUSE_CLIENT --query="SELECT * FROM custom_separated ORDER BY n FORMAT CSV"
$CLICKHOUSE_CLIENT --query="DROP TABLE custom_separated"
echo -ne "a,b\nc,d\n" | $CLICKHOUSE_LOCAL --structure "a String, b String" \
--input-format CustomSeparated --format_custom_escaping_rule=Escaped \
--format_custom_field_delimiter=',' --format_custom_row_after_delimiter=$'\n' -q 'select * from table' \
2>&1| grep -Fac "'Escaped' serialization requires delimiter"