mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
read settings from file
This commit is contained in:
parent
769d33848b
commit
e197cc8a49
@ -217,8 +217,9 @@ struct Settings : public SettingsCollection<Settings>
|
||||
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_schema_rows, "", "Row format string for Template format") \
|
||||
M(SettingString, format_schema_rows_between_delimiter, "\n", "Delimiter between rows for Template format") \
|
||||
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(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") \
|
||||
|
@ -48,9 +48,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings)
|
||||
format_settings.date_time_input_format = settings.date_time_input_format;
|
||||
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
|
||||
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
|
||||
format_settings.template_settings.format = settings.format_schema;
|
||||
format_settings.template_settings.row_format = settings.format_schema_rows;
|
||||
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
|
||||
format_settings.template_settings.resultset_format = settings.format_template_resultset;
|
||||
format_settings.template_settings.row_format = settings.format_template_row;
|
||||
format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter;
|
||||
|
||||
return format_settings;
|
||||
}
|
||||
@ -67,9 +67,9 @@ static FormatSettings getOutputFormatSetting(const Settings & settings)
|
||||
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
|
||||
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
|
||||
format_settings.pretty.color = settings.output_format_pretty_color;
|
||||
format_settings.template_settings.format = settings.format_schema;
|
||||
format_settings.template_settings.row_format = settings.format_schema_rows;
|
||||
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
|
||||
format_settings.template_settings.resultset_format = settings.format_template_resultset;
|
||||
format_settings.template_settings.row_format = settings.format_template_row;
|
||||
format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter;
|
||||
format_settings.write_statistics = settings.output_format_write_statistics;
|
||||
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
|
||||
|
||||
|
@ -26,28 +26,33 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & format)
|
||||
FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & format_schema, const String & format, bool require_message)
|
||||
{
|
||||
String format_schema = context.getSettingsRef().format_schema.toString();
|
||||
if (format_schema.empty())
|
||||
throw Exception(
|
||||
"The format " + format + " requires a schema. The 'format_schema' setting should be set", ErrorCodes::BAD_ARGUMENTS);
|
||||
"The format " + format + " requires a schema. The corresponding setting should be set", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String default_file_extension = getFormatSchemaDefaultFileExtension(format);
|
||||
|
||||
size_t colon_pos = format_schema.find(':');
|
||||
Poco::Path path;
|
||||
if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)
|
||||
|| path.assign(format_schema.substr(0, colon_pos)).makeFile().getFileName().empty())
|
||||
if (require_message)
|
||||
{
|
||||
throw Exception(
|
||||
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
|
||||
+ (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + ". Got '" + format_schema
|
||||
+ "'",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
size_t colon_pos = format_schema.find(':');
|
||||
if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)
|
||||
|| path.assign(format_schema.substr(0, colon_pos)).makeFile().getFileName().empty())
|
||||
{
|
||||
throw Exception(
|
||||
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
|
||||
+ (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") +
|
||||
". Got '" + format_schema
|
||||
+ "'",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
message_name = format_schema.substr(colon_pos + 1);
|
||||
message_name = format_schema.substr(colon_pos + 1);
|
||||
}
|
||||
else
|
||||
path.assign(format_schema).makeFile().getFileName();
|
||||
|
||||
auto default_schema_directory = [&context]()
|
||||
{
|
||||
|
@ -10,7 +10,7 @@ class Context;
|
||||
class FormatSchemaInfo
|
||||
{
|
||||
public:
|
||||
FormatSchemaInfo(const Context & context, const String & format);
|
||||
FormatSchemaInfo(const Context & context, const String & format_schema, const String & format, bool require_message);
|
||||
|
||||
/// Returns path to the schema file.
|
||||
const String & schemaPath() const { return schema_path; }
|
||||
|
@ -52,7 +52,7 @@ struct FormatSettings
|
||||
|
||||
struct Template
|
||||
{
|
||||
String format;
|
||||
String resultset_format;
|
||||
String row_format;
|
||||
String row_between_delimiter;
|
||||
};
|
||||
|
@ -2,6 +2,8 @@
|
||||
#include <Formats/verbosePrintString.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,11 +13,14 @@ namespace ErrorCodes
|
||||
extern const int INVALID_TEMPLATE_FORMAT;
|
||||
}
|
||||
|
||||
ParsedTemplateFormatString::ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name)
|
||||
ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name)
|
||||
{
|
||||
try
|
||||
{
|
||||
parse(format_string, idx_by_name);
|
||||
ReadBufferFromFile schema_file(schema.absoluteSchemaPath());
|
||||
WriteBufferFromOwnString format_string;
|
||||
copyData(schema_file, format_string);
|
||||
parse(format_string.str(), idx_by_name);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <Formats/FormatSchemaInfo.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,7 +35,7 @@ struct ParsedTemplateFormatString
|
||||
typedef std::function<std::optional<size_t>(const String &)> ColumnIdxGetter;
|
||||
|
||||
ParsedTemplateFormatString() = default;
|
||||
ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name);
|
||||
ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name);
|
||||
|
||||
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name);
|
||||
|
||||
|
@ -303,7 +303,8 @@ void registerInputFormatProcessorCapnProto(FormatFactory & factory)
|
||||
"CapnProto",
|
||||
[](ReadBuffer & buf, const Block & sample, const Context & context, IRowInputFormat::Params params, const FormatSettings &)
|
||||
{
|
||||
return std::make_shared<CapnProtoRowInputFormat>(buf, sample, std::move(params), FormatSchemaInfo(context, "CapnProto"));
|
||||
return std::make_shared<CapnProtoRowInputFormat>(buf, sample, std::move(params),
|
||||
FormatSchemaInfo(context, context.getSettingsRef().format_schema, "CapnProto", true));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/FormatSchemaInfo.h>
|
||||
#include <Formats/ProtobufSchemas.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -74,7 +75,8 @@ void registerInputFormatProcessorProtobuf(FormatFactory & factory)
|
||||
IRowInputFormat::Params params,
|
||||
const FormatSettings &)
|
||||
{
|
||||
return std::make_shared<ProtobufRowInputFormat>(buf, sample, std::move(params), FormatSchemaInfo(context, "Protobuf"));
|
||||
return std::make_shared<ProtobufRowInputFormat>(buf, sample, std::move(params),
|
||||
FormatSchemaInfo(context, context.getSettingsRef().format_schema, "Protobuf", true));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -54,7 +54,8 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory)
|
||||
FormatFactory::WriteCallback callback,
|
||||
const FormatSettings &)
|
||||
{
|
||||
return std::make_shared<ProtobufRowOutputFormat>(buf, header, callback, FormatSchemaInfo(context, "Protobuf"));
|
||||
return std::make_shared<ProtobufRowOutputFormat>(buf, header, callback,
|
||||
FormatSchemaInfo(context, context.getSettingsRef().format_schema, "Protobuf", true));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -12,8 +12,9 @@ namespace ErrorCodes
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_)
|
||||
: IOutputFormat(header_, out_), settings(settings_)
|
||||
TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_,
|
||||
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_)
|
||||
: IOutputFormat(header_, out_), settings(settings_), format(std::move(format_)), row_format(std::move(row_format_))
|
||||
{
|
||||
auto & sample = getPort(PortKind::Main).getHeader();
|
||||
size_t columns = sample.columns();
|
||||
@ -21,14 +22,6 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
types[i] = sample.safeGetByPosition(i).type;
|
||||
|
||||
/// Parse format string for whole output
|
||||
static const String default_format("${data}");
|
||||
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
|
||||
format = ParsedTemplateFormatString(format_str, [&](const String & partName)
|
||||
{
|
||||
return static_cast<size_t>(stringToOutputPart(partName));
|
||||
});
|
||||
|
||||
/// Validate format string for whole output
|
||||
size_t data_idx = format.format_idx_to_column_idx.size() + 1;
|
||||
for (size_t i = 0; i < format.format_idx_to_column_idx.size(); ++i)
|
||||
@ -37,20 +30,20 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ
|
||||
format.throwInvalidFormat("Output part name cannot be empty.", i);
|
||||
switch (*format.format_idx_to_column_idx[i])
|
||||
{
|
||||
case static_cast<size_t>(OutputPart::Data):
|
||||
case static_cast<size_t>(ResultsetPart::Data):
|
||||
data_idx = i;
|
||||
[[fallthrough]];
|
||||
case static_cast<size_t>(OutputPart::Totals):
|
||||
case static_cast<size_t>(OutputPart::ExtremesMin):
|
||||
case static_cast<size_t>(OutputPart::ExtremesMax):
|
||||
case static_cast<size_t>(ResultsetPart::Totals):
|
||||
case static_cast<size_t>(ResultsetPart::ExtremesMin):
|
||||
case static_cast<size_t>(ResultsetPart::ExtremesMax):
|
||||
if (format.formats[i] != ColumnFormat::None)
|
||||
format.throwInvalidFormat("Serialization type for data, totals, min and max must be empty or None", i);
|
||||
break;
|
||||
case static_cast<size_t>(OutputPart::Rows):
|
||||
case static_cast<size_t>(OutputPart::RowsBeforeLimit):
|
||||
case static_cast<size_t>(OutputPart::TimeElapsed):
|
||||
case static_cast<size_t>(OutputPart::RowsRead):
|
||||
case static_cast<size_t>(OutputPart::BytesRead):
|
||||
case static_cast<size_t>(ResultsetPart::Rows):
|
||||
case static_cast<size_t>(ResultsetPart::RowsBeforeLimit):
|
||||
case static_cast<size_t>(ResultsetPart::TimeElapsed):
|
||||
case static_cast<size_t>(ResultsetPart::RowsRead):
|
||||
case static_cast<size_t>(ResultsetPart::BytesRead):
|
||||
if (format.formats[i] == ColumnFormat::None)
|
||||
format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, "
|
||||
"rows_read or bytes_read is not specified", i);
|
||||
@ -62,12 +55,6 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ
|
||||
if (data_idx != 0)
|
||||
format.throwInvalidFormat("${data} must be the first output part", 0);
|
||||
|
||||
/// Parse format string for rows
|
||||
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName)
|
||||
{
|
||||
return sample.getPositionByName(colName);
|
||||
});
|
||||
|
||||
/// Validate format string for rows
|
||||
if (row_format.delimiters.size() == 1)
|
||||
row_format.throwInvalidFormat("No columns specified", 0);
|
||||
@ -83,26 +70,26 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ
|
||||
}
|
||||
}
|
||||
|
||||
TemplateBlockOutputFormat::OutputPart TemplateBlockOutputFormat::stringToOutputPart(const String & part)
|
||||
TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResultsetPart(const String & part)
|
||||
{
|
||||
if (part == "data")
|
||||
return OutputPart::Data;
|
||||
return ResultsetPart::Data;
|
||||
else if (part == "totals")
|
||||
return OutputPart::Totals;
|
||||
return ResultsetPart::Totals;
|
||||
else if (part == "min")
|
||||
return OutputPart::ExtremesMin;
|
||||
return ResultsetPart::ExtremesMin;
|
||||
else if (part == "max")
|
||||
return OutputPart::ExtremesMax;
|
||||
return ResultsetPart::ExtremesMax;
|
||||
else if (part == "rows")
|
||||
return OutputPart::Rows;
|
||||
return ResultsetPart::Rows;
|
||||
else if (part == "rows_before_limit")
|
||||
return OutputPart::RowsBeforeLimit;
|
||||
return ResultsetPart::RowsBeforeLimit;
|
||||
else if (part == "time")
|
||||
return OutputPart::TimeElapsed;
|
||||
return ResultsetPart::TimeElapsed;
|
||||
else if (part == "rows_read")
|
||||
return OutputPart::RowsRead;
|
||||
return ResultsetPart::RowsRead;
|
||||
else if (part == "bytes_read")
|
||||
return OutputPart::BytesRead;
|
||||
return ResultsetPart::BytesRead;
|
||||
else
|
||||
throw Exception("Unknown output part " + part, ErrorCodes::SYNTAX_ERROR);
|
||||
}
|
||||
@ -193,38 +180,38 @@ void TemplateBlockOutputFormat::finalize()
|
||||
{
|
||||
auto type = std::make_shared<DataTypeUInt64>();
|
||||
ColumnWithTypeAndName col(type->createColumnConst(1, row_count), type, String("tmp"));
|
||||
switch (static_cast<OutputPart>(*format.format_idx_to_column_idx[i]))
|
||||
switch (static_cast<ResultsetPart>(*format.format_idx_to_column_idx[i]))
|
||||
{
|
||||
case OutputPart::Totals:
|
||||
case ResultsetPart::Totals:
|
||||
if (!totals)
|
||||
format.throwInvalidFormat("Cannot print totals for this request", i);
|
||||
writeRow(totals, 0);
|
||||
break;
|
||||
case OutputPart::ExtremesMin:
|
||||
case ResultsetPart::ExtremesMin:
|
||||
if (!extremes)
|
||||
format.throwInvalidFormat("Cannot print extremes for this request", i);
|
||||
writeRow(extremes, 0);
|
||||
break;
|
||||
case OutputPart::ExtremesMax:
|
||||
case ResultsetPart::ExtremesMax:
|
||||
if (!extremes)
|
||||
format.throwInvalidFormat("Cannot print extremes for this request", i);
|
||||
writeRow(extremes, 1);
|
||||
break;
|
||||
case OutputPart::Rows:
|
||||
case ResultsetPart::Rows:
|
||||
writeValue<size_t, DataTypeUInt64>(row_count, format.formats[i]);
|
||||
break;
|
||||
case OutputPart::RowsBeforeLimit:
|
||||
case ResultsetPart::RowsBeforeLimit:
|
||||
if (!rows_before_limit_set)
|
||||
format.throwInvalidFormat("Cannot print rows_before_limit for this request", i);
|
||||
writeValue<size_t, DataTypeUInt64>(rows_before_limit, format.formats[i]);
|
||||
break;
|
||||
case OutputPart::TimeElapsed:
|
||||
case ResultsetPart::TimeElapsed:
|
||||
writeValue<double, DataTypeFloat64>(watch.elapsedSeconds(), format.formats[i]);
|
||||
break;
|
||||
case OutputPart::RowsRead:
|
||||
case ResultsetPart::RowsRead:
|
||||
writeValue<size_t, DataTypeUInt64>(progress.read_rows.load(), format.formats[i]);
|
||||
break;
|
||||
case OutputPart::BytesRead:
|
||||
case ResultsetPart::BytesRead:
|
||||
writeValue<size_t, DataTypeUInt64>(progress.read_bytes.load(), format.formats[i]);
|
||||
break;
|
||||
default:
|
||||
@ -242,11 +229,38 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory)
|
||||
factory.registerOutputFormatProcessor("Template", [](
|
||||
WriteBuffer & buf,
|
||||
const Block & sample,
|
||||
const Context &,
|
||||
const Context & context,
|
||||
FormatFactory::WriteCallback,
|
||||
const FormatSettings & settings)
|
||||
{
|
||||
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, settings);
|
||||
ParsedTemplateFormatString resultset_format;
|
||||
if (settings.template_settings.resultset_format.empty())
|
||||
{
|
||||
/// Default format string: "${data}"
|
||||
resultset_format.delimiters.resize(2);
|
||||
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");
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Read format string from file
|
||||
resultset_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(context, settings.template_settings.resultset_format, "Template", false),
|
||||
[&](const String & partName)
|
||||
{
|
||||
return static_cast<size_t>(TemplateBlockOutputFormat::stringToResultsetPart(partName));
|
||||
});
|
||||
}
|
||||
|
||||
ParsedTemplateFormatString row_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(context, settings.template_settings.row_format, "Template", false),
|
||||
[&](const String & colName)
|
||||
{
|
||||
return sample.getPositionByName(colName);
|
||||
});
|
||||
|
||||
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, settings, resultset_format, row_format);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -14,7 +14,8 @@ class TemplateBlockOutputFormat : public IOutputFormat
|
||||
{
|
||||
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
|
||||
public:
|
||||
TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_);
|
||||
TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_,
|
||||
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_);
|
||||
|
||||
String getName() const override { return "TemplateBlockOutputFormat"; }
|
||||
|
||||
@ -23,13 +24,7 @@ public:
|
||||
void setRowsBeforeLimit(size_t rows_before_limit_) override { rows_before_limit = rows_before_limit_; rows_before_limit_set = true; }
|
||||
void onProgress(const Progress & progress_) override { progress.incrementPiecewiseAtomically(progress_); }
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override;
|
||||
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
|
||||
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
|
||||
void finalize() override;
|
||||
|
||||
enum class OutputPart : size_t
|
||||
enum class ResultsetPart : size_t
|
||||
{
|
||||
Data,
|
||||
Totals,
|
||||
@ -42,7 +37,14 @@ protected:
|
||||
BytesRead
|
||||
};
|
||||
|
||||
OutputPart stringToOutputPart(const String & part);
|
||||
static ResultsetPart stringToResultsetPart(const String & part);
|
||||
|
||||
protected:
|
||||
void consume(Chunk chunk) override;
|
||||
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
|
||||
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
|
||||
void finalize() override;
|
||||
|
||||
void writeRow(const Chunk & chunk, size_t row_num);
|
||||
void serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat format);
|
||||
template <typename U, typename V> void writeValue(U value, ColumnFormat col_format);
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Formats/verbosePrintString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -18,21 +19,13 @@ extern const int SYNTAX_ERROR;
|
||||
|
||||
|
||||
TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
|
||||
const FormatSettings & settings_, bool ignore_spaces_)
|
||||
const FormatSettings & settings_, bool ignore_spaces_,
|
||||
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_)
|
||||
: RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()),
|
||||
settings(settings_), ignore_spaces(ignore_spaces_)
|
||||
settings(settings_), ignore_spaces(ignore_spaces_),
|
||||
format(std::move(format_)), row_format(std::move(row_format_))
|
||||
{
|
||||
/// Parse format string for whole input
|
||||
static const String default_format("${data}");
|
||||
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
|
||||
format = ParsedTemplateFormatString(format_str, [&](const String & partName) -> std::optional<size_t>
|
||||
{
|
||||
if (partName == "data")
|
||||
return 0;
|
||||
throw Exception("Unknown input part " + partName, ErrorCodes::SYNTAX_ERROR);
|
||||
});
|
||||
|
||||
/// Validate format string for whole input
|
||||
/// Validate format string for result set
|
||||
bool has_data = false;
|
||||
for (size_t i = 0; i < format.columnsCount(); ++i)
|
||||
{
|
||||
@ -54,12 +47,6 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer
|
||||
}
|
||||
}
|
||||
|
||||
/// Parse format string for rows
|
||||
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName) -> std::optional<size_t>
|
||||
{
|
||||
return header_.getPositionByName(colName);
|
||||
});
|
||||
|
||||
/// Validate format string for rows
|
||||
std::vector<UInt8> column_in_format(header_.columns(), false);
|
||||
for (size_t i = 0; i < row_format.columnsCount(); ++i)
|
||||
@ -494,11 +481,41 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory)
|
||||
factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=](
|
||||
ReadBuffer & buf,
|
||||
const Block & sample,
|
||||
const Context &,
|
||||
const Context & context,
|
||||
IRowInputFormat::Params params,
|
||||
const FormatSettings & settings)
|
||||
{
|
||||
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces);
|
||||
ParsedTemplateFormatString resultset_format;
|
||||
if (settings.template_settings.resultset_format.empty())
|
||||
{
|
||||
/// Default format string: "${data}"
|
||||
resultset_format.delimiters.resize(2);
|
||||
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");
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Read format string from file
|
||||
resultset_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(context, settings.template_settings.resultset_format, "Template", false),
|
||||
[&](const String & partName) -> std::optional<size_t>
|
||||
{
|
||||
if (partName == "data")
|
||||
return 0;
|
||||
throw Exception("Unknown input part " + partName,
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
});
|
||||
}
|
||||
|
||||
ParsedTemplateFormatString row_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(context, settings.template_settings.row_format, "Template", false),
|
||||
[&](const String & colName) -> std::optional<size_t>
|
||||
{
|
||||
return sample.getPositionByName(colName);
|
||||
});
|
||||
|
||||
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces, resultset_format, row_format);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -16,7 +16,8 @@ class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
|
||||
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
|
||||
public:
|
||||
TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
|
||||
const FormatSettings & settings_, bool ignore_spaces_);
|
||||
const FormatSettings & settings_, bool ignore_spaces_,
|
||||
ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_);
|
||||
|
||||
String getName() const override { return "TemplateRowInputFormat"; }
|
||||
|
||||
@ -50,9 +51,9 @@ private:
|
||||
DataTypes data_types;
|
||||
|
||||
FormatSettings settings;
|
||||
const bool ignore_spaces;
|
||||
ParsedTemplateFormatString format;
|
||||
ParsedTemplateFormatString row_format;
|
||||
const bool ignore_spaces;
|
||||
|
||||
size_t format_data_idx;
|
||||
bool end_of_stream = false;
|
||||
|
22
dbms/tests/queries/0_stateless/00937_template_output_format.sh
Executable file
22
dbms/tests/queries/0_stateless/00937_template_output_format.sh
Executable file
@ -0,0 +1,22 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE template (s1 String, s2 String, \`s 3\` String, \"s 4\" String, n UInt64, d Date) ENGINE = Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO template VALUES
|
||||
('qwe,rty', 'as\"df''gh', '', 'zx\ncv\tbn m', 123, '2016-01-01'),\
|
||||
('as\"df''gh', '', 'zx\ncv\tbn m', 'qwe,rty', 456, '2016-01-02'),\
|
||||
('', 'zx\ncv\tbn m', 'qwe,rty', 'as\"df''gh', 9876543210, '2016-01-03'),\
|
||||
('zx\ncv\tbn m', 'qwe,rty', 'as\"df''gh', '', 789, '2016-01-04')";
|
||||
|
||||
echo -ne '{prefix} \n${data:None}\n------\n${totals:}\n------\n${min}\n------\n${max}\n${rows:Escaped} rows\nbefore limit ${rows_before_limit:XML}\nread ${rows_read:Escaped} $$ suffix $$' > $CURDIR/00937_template_output_format_resultset.tmp
|
||||
echo -ne 'n:\t${n:JSON}, s1:\t${0:Escaped}, s2:\t${s2:Quoted}, s3:\t${`s 3`:JSON}, s4:\t${"s 4":CSV}, d:\t${d:Escaped}, n:\t${n:Raw}\t' > $CURDIR/00937_template_output_format_row.tmp
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template GROUP BY s1, s2, \`s 3\`, \"s 4\", n, d WITH TOTALS ORDER BY n LIMIT 4 FORMAT Template SETTINGS extremes = 1,\
|
||||
format_template_resultset = '$CURDIR/00937_template_output_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00937_template_output_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ';\n'";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE template";
|
@ -1,12 +0,0 @@
|
||||
DROP TABLE IF EXISTS template;
|
||||
CREATE TABLE template (s1 String, s2 String, `s 3` String, "s 4" String, n UInt64, d Date) ENGINE = Memory;
|
||||
INSERT INTO template VALUES
|
||||
('qwe,rty', 'as"df''gh', '', 'zx\ncv\tbn m', 123, '2016-01-01'),('as"df''gh', '', 'zx\ncv\tbn m', 'qwe,rty', 456, '2016-01-02'),('', 'zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', 9876543210, '2016-01-03'),('zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', '', 789, '2016-01-04');
|
||||
|
||||
SELECT * FROM template GROUP BY s1, s2, `s 3`, "s 4", n, d WITH TOTALS ORDER BY n LIMIT 4 FORMAT Template SETTINGS
|
||||
extremes = 1,
|
||||
format_schema = '{prefix} \n${data:None}\n------\n${totals:}\n------\n${min}\n------\n${max}\n${rows:Escaped} rows\nbefore limit ${rows_before_limit:XML}\nread ${rows_read:Escaped} $$ suffix $$',
|
||||
format_schema_rows = 'n:\t${n:JSON}, s1:\t${0:Escaped}, s2:\t${s2:Quoted}, s3:\t${`s 3`:JSON}, s4:\t${"s 4":CSV}, d:\t${d:Escaped}, n:\t${n:Raw}\t',
|
||||
format_schema_rows_between_delimiter = ';\n';
|
||||
|
||||
DROP TABLE template;
|
@ -9,6 +9,8 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE template1 (s1 String, s2 String, s3 Str
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE template2 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory";
|
||||
|
||||
echo "==== check escaping ===="
|
||||
echo -ne '{prefix} \n${data}\n $$ suffix $$\n' > $CURDIR/00938_template_input_format_resultset.tmp
|
||||
echo -ne 'n:\t${n:Escaped}, s1:\t${0:Escaped}\t, s2:\t${1:Quoted}, s3:\t${s3:JSON}, s4:\t${3:CSV}, d:\t${d:Escaped}\t' > $CURDIR/00938_template_input_format_row.tmp
|
||||
|
||||
echo "{prefix}
|
||||
n: 123, s1: qwe,rty , s2: 'as\"df\\'gh', s3: \"\", s4: \"zx
|
||||
@ -17,28 +19,32 @@ n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 20
|
||||
n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ;
|
||||
n: 789, s1: zx\\ncv\\tbn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04
|
||||
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \
|
||||
format_schema = '{prefix} \n\${data}\n \$\$ suffix \$\$\n', \
|
||||
format_schema_rows = 'n:\t\${n:Escaped}, s1:\t\${0:Escaped}\t, s2:\t\${1:Quoted}, s3:\t\${s3:JSON}, s4:\t\${3:CSV}, d:\t\${d:Escaped}\t', \
|
||||
format_schema_rows_between_delimiter = ';\n'";
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ';\n'";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV";
|
||||
|
||||
echo "==== parse json (sophisticated template) ===="
|
||||
echo -ne '{${:}"meta"${:}:${:}[${:}{${:}"name"${:}:${:}"s1"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s2"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s3"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s4"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"n"${:},${:}"type"${:}:${:}"UInt64"${:}}${:},${:}{${:}"name"${:}:${:}"d"${:},${:}"type"${:}:${:}"Date"${:}}${:}]${:},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:}${:CSV}${:},${:}"statistics"${:}:${:}{${:}"elapsed"${:}:${:}${:CSV}${:},${:}"rows_read"${:}:${:}${:CSV}${:},${:}"bytes_read"${:}:${:}${:CSV}${:}}${:}}' > $CURDIR/00938_template_input_format_resultset.tmp
|
||||
echo -ne '{${:}"s1"${:}:${:}${s1:JSON}${:},${:}"s2"${:}:${:}${s2:JSON}${:},${:}"s3"${:}:${:}${s3:JSON}${:},${:}"s4"${:}:${:}${s4:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > $CURDIR/00938_template_input_format_row.tmp
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
|
||||
format_schema = '{\${:}\"meta\"\${:}:\${:}[\${:}{\${:}\"name\"\${:}:\${:}\"s1\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s2\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s3\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s4\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"n\"\${:},\${:}\"type\"\${:}:\${:}\"UInt64\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"d\"\${:},\${:}\"type\"\${:}:\${:}\"Date\"\${:}}\${:}]\${:},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:}\${:CSV}\${:},\${:}\"statistics\"\${:}:\${:}{\${:}\"elapsed\"\${:}:\${:}\${:CSV}\${:},\${:}\"rows_read\"\${:}:\${:}\${:CSV}\${:},\${:}\"bytes_read\"\${:}:\${:}\${:CSV}\${:}}\${:}}', \
|
||||
format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${s2:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${s4:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \
|
||||
format_schema_rows_between_delimiter = ','";
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ','";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
|
||||
$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template2";
|
||||
|
||||
echo "==== parse json ===="
|
||||
|
||||
echo -ne '{${:}"meta"${:}:${:JSON},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:JSON},${:}"statistics"${:}:${:JSON}${:}}' > $CURDIR/00938_template_input_format_resultset.tmp
|
||||
echo -ne '{${:}"s1"${:}:${:}${s3:JSON}${:},${:}"s2"${:}:${:}${:JSON}${:},${:}"s3"${:}:${:}${s1:JSON}${:},${:}"s4"${:}:${:}${:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > $CURDIR/00938_template_input_format_row.tmp
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
|
||||
format_schema = '{\${:}\"meta\"\${:}:\${:JSON},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:JSON},\${:}\"statistics\"\${:}:\${:JSON}\${:}}', \
|
||||
format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \
|
||||
format_schema_rows_between_delimiter = ','";
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ','";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user