add CustomSeparated format

This commit is contained in:
Alexander Tokmakov 2019-09-24 22:56:45 +03:00
parent e197cc8a49
commit d642304b1d
8 changed files with 132 additions and 4 deletions

View File

@ -216,10 +216,20 @@ struct Settings : public SettingsCollection<Settings>
M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.") \
M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \
M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \
\
M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \
M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)") \
M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)") \
M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)") \
\
M(SettingString, format_custom_escaping_rule, "", "Field escaping rule (for CustomSeparated format)") \
M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)") \
M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)") \
M(SettingString, format_custom_row_after_delimiter, "", "Delimiter after field of the last column (for CustomSeparated format)") \
M(SettingString, format_custom_row_between_delimiter, "\n", "Delimiter between rows (for CustomSeparated format)") \
M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)") \
M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)") \
\
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \

View File

@ -4,6 +4,8 @@
#include <IO/Operators.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/copyData.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -119,7 +121,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum
}
ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format) const
ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format)
{
if (col_format.empty())
return ColumnFormat::None;
@ -138,7 +140,7 @@ ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFor
else if (col_format == "Raw")
return ColumnFormat::Raw;
else
throwInvalidFormat("Unknown field format " + col_format, columnsCount());
throw Exception("Unknown field format \"" + col_format + "\"", ErrorCodes::BAD_ARGUMENTS);
}
size_t ParsedTemplateFormatString::columnsCount() const
@ -233,4 +235,38 @@ void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size
ErrorCodes::INVALID_TEMPLATE_FORMAT);
}
ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(const Context & context)
{
const Settings & settings = context.getSettingsRef();
/// Set resultset format to "result_before_delimiter ${data} result_after_delimiter"
ParsedTemplateFormatString resultset_format;
resultset_format.delimiters.emplace_back(settings.format_custom_result_before_delimiter);
resultset_format.delimiters.emplace_back(settings.format_custom_result_after_delimiter);
resultset_format.formats.emplace_back(ParsedTemplateFormatString::ColumnFormat::None);
resultset_format.format_idx_to_column_idx.emplace_back(0);
resultset_format.column_names.emplace_back("data");
return resultset_format;
}
ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedRowFormat(const Context & context, const Block & sample)
{
const Settings & settings = context.getSettingsRef();
/// Set row format to
/// "row_before_delimiter ${Col0:escaping} field_delimiter ${Col1:escaping} field_delimiter ... ${ColN:escaping} row_after_delimiter"
ParsedTemplateFormatString::ColumnFormat escaping = ParsedTemplateFormatString::stringToFormat(settings.format_custom_escaping_rule);
ParsedTemplateFormatString row_format;
row_format.delimiters.emplace_back(settings.format_custom_row_before_delimiter);
for (size_t i = 0; i < sample.columns(); ++i)
{
row_format.formats.emplace_back(escaping);
row_format.format_idx_to_column_idx.emplace_back(i);
row_format.column_names.emplace_back(sample.getByPosition(i).name);
bool last_column = i == sample.columns() - 1;
row_format.delimiters.emplace_back(last_column ? settings.format_custom_row_after_delimiter : settings.format_custom_field_delimiter);
}
return row_format;
}
}

View File

@ -8,6 +8,8 @@
namespace DB
{
class Block;
struct ParsedTemplateFormatString
{
enum class ColumnFormat
@ -39,13 +41,16 @@ struct ParsedTemplateFormatString
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name);
ColumnFormat stringToFormat(const String & format) const;
static ColumnFormat stringToFormat(const String & format);
static String formatToString(ColumnFormat format);
static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s);
size_t columnsCount() const;
String dump() const;
[[noreturn]] void throwInvalidFormat(const String & message, size_t column) const;
static ParsedTemplateFormatString setupCustomSeparatedResultsetFormat(const Context & context);
static ParsedTemplateFormatString setupCustomSeparatedRowFormat(const Context & context, const Block & sample);
};
}

View File

@ -2,6 +2,7 @@
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
namespace DB
@ -262,5 +263,20 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory)
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, settings, resultset_format, row_format);
});
factory.registerOutputFormatProcessor("CustomSeparated", [](
WriteBuffer & buf,
const Block & sample,
const Context & context,
FormatFactory::WriteCallback,
const FormatSettings & settings)
{
ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(context);
ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(context, sample);
FormatSettings format_settings = settings;
format_settings.template_settings.row_between_delimiter = context.getSettingsRef().format_custom_row_between_delimiter;
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, format_settings, resultset_format, row_format);
});
}
}

View File

@ -518,6 +518,24 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory)
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces, resultset_format, row_format);
});
}
for (bool ignore_spaces : {false, true})
{
factory.registerInputFormatProcessor(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", [=](
ReadBuffer & buf,
const Block & sample,
const Context & context,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(context);
ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(context, sample);
FormatSettings format_settings = settings;
format_settings.template_settings.row_between_delimiter = context.getSettingsRef().format_custom_row_between_delimiter;
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, format_settings, ignore_spaces, resultset_format, row_format);
});
}
}
}

View File

@ -50,7 +50,7 @@ private:
PeekableReadBuffer buf;
DataTypes data_types;
FormatSettings settings;
const FormatSettings settings;
const bool ignore_spaces;
ParsedTemplateFormatString format;
ParsedTemplateFormatString row_format;

View File

@ -0,0 +1,10 @@
========== result ==========
||0 | "2019-09-24" | "hello" ||
||1 | "2019-09-25" | "world" ||
||2 | "2019-09-26" | "custom" ||
||3 | "2019-09-27" | "separated" ||
============================
0,"2019-09-24","hello"
1,"2019-09-25","world"
2,"2019-09-26","custom"
3,"2019-09-27","separated"

View File

@ -0,0 +1,33 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS custom_separated"
$CLICKHOUSE_CLIENT --query="CREATE TABLE custom_separated (n UInt64, d Date, s String) ENGINE = Memory()"
$CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated VALUES (0, '2019-09-24', 'hello'), (1, '2019-09-25', 'world'), (2, '2019-09-26', 'custom'), (3, '2019-09-27', 'separated')"
$CLICKHOUSE_CLIENT --query="SELECT * FROM custom_separated ORDER BY n FORMAT CustomSeparated SETTINGS \
format_custom_escaping_rule = 'CSV', \
format_custom_field_delimiter = '\t|\t', \
format_custom_row_before_delimiter = '||', \
format_custom_row_after_delimiter = '\t||', \
format_custom_result_before_delimiter = '========== result ==========\n', \
format_custom_result_after_delimiter = '\n============================\n'"
$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE custom_separated"
echo '0, "2019-09-24", "hello"
1, 2019-09-25, "world"
2, "2019-09-26", custom
3, 2019-09-27, separated
end' | $CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated FORMAT CustomSeparated SETTINGS \
format_custom_escaping_rule = 'CSV', \
format_custom_field_delimiter = ', ', \
format_custom_row_after_delimiter = '\n', \
format_custom_row_between_delimiter = '', \
format_custom_result_after_delimiter = 'end\n'"
$CLICKHOUSE_CLIENT --query="SELECT * FROM custom_separated ORDER BY n FORMAT CSV"
$CLICKHOUSE_CLIENT --query="DROP TABLE custom_separated"