2020-04-16 12:31:57 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
|
|
|
|
2019-10-28 23:43:22 +00:00
|
|
|
#include <algorithm>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2019-03-22 12:08:30 +00:00
|
|
|
#include <Core/Settings.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <DataStreams/MaterializingBlockOutputStream.h>
|
2019-10-01 10:48:46 +00:00
|
|
|
#include <DataStreams/ParallelParsingBlockInputStream.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2019-02-19 18:41:18 +00:00
|
|
|
#include <Processors/Formats/IRowInputFormat.h>
|
2019-07-24 18:00:09 +00:00
|
|
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
|
|
|
#include <Processors/Formats/OutputStreamToOutputFormat.h>
|
2019-07-31 13:26:08 +00:00
|
|
|
#include <DataStreams/NativeBlockInputStream.h>
|
2019-12-25 19:17:41 +00:00
|
|
|
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
|
|
|
|
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
|
2020-05-30 17:04:02 +00:00
|
|
|
#include <Processors/Formats/Impl/PostgreSQLOutputFormat.h>
|
2020-02-02 00:53:11 +00:00
|
|
|
#include <Poco/URI.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include <Common/config.h>
|
|
|
|
#endif
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int UNKNOWN_FORMAT;
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int FORMAT_IS_NOT_SUITABLE_FOR_INPUT;
|
|
|
|
extern const int FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT;
|
|
|
|
}
|
|
|
|
|
|
|
|
const FormatFactory::Creators & FormatFactory::getCreators(const String & name) const
|
|
|
|
{
|
|
|
|
auto it = dict.find(name);
|
|
|
|
if (dict.end() != it)
|
|
|
|
return it->second;
|
|
|
|
throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
static FormatSettings getInputFormatSetting(const Settings & settings, const Context & context)
|
2019-02-19 18:41:18 +00:00
|
|
|
{
|
2018-06-10 19:22:49 +00:00
|
|
|
FormatSettings format_settings;
|
|
|
|
format_settings.csv.delimiter = settings.format_csv_delimiter;
|
2018-07-04 21:00:50 +00:00
|
|
|
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
|
|
|
|
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
|
2019-07-22 20:13:27 +00:00
|
|
|
format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null;
|
2019-06-20 12:46:36 +00:00
|
|
|
format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields;
|
2019-10-04 17:19:49 +00:00
|
|
|
format_settings.null_as_default = settings.input_format_null_as_default;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
|
2019-05-23 01:30:00 +00:00
|
|
|
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
|
2019-09-04 19:42:01 +00:00
|
|
|
format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals;
|
2019-04-22 13:31:17 +00:00
|
|
|
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
|
2018-09-14 13:43:57 +00:00
|
|
|
format_settings.import_nested_json = settings.input_format_import_nested_json;
|
2018-06-10 19:22:49 +00:00
|
|
|
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;
|
2019-09-24 14:25:22 +00:00
|
|
|
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;
|
2019-10-16 14:22:22 +00:00
|
|
|
format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default;
|
2019-12-25 19:17:41 +00:00
|
|
|
format_settings.schema.format_schema = settings.format_schema;
|
|
|
|
format_settings.schema.format_schema_path = context.getFormatSchemaPath();
|
|
|
|
format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER);
|
|
|
|
format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter;
|
|
|
|
format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter;
|
|
|
|
format_settings.custom.escaping_rule = settings.format_custom_escaping_rule;
|
|
|
|
format_settings.custom.field_delimiter = settings.format_custom_field_delimiter;
|
|
|
|
format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter;
|
|
|
|
format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter;
|
|
|
|
format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter;
|
2020-02-14 19:48:45 +00:00
|
|
|
format_settings.regexp.regexp = settings.format_regexp;
|
|
|
|
format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule;
|
|
|
|
format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched;
|
2020-02-02 00:53:11 +00:00
|
|
|
|
|
|
|
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
|
|
|
|
if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER))
|
|
|
|
{
|
|
|
|
const Poco::URI & avro_schema_registry_url = settings.format_avro_schema_registry_url;
|
|
|
|
if (!avro_schema_registry_url.empty())
|
|
|
|
context.getRemoteHostFilter().checkURL(avro_schema_registry_url);
|
|
|
|
}
|
|
|
|
format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString();
|
2020-06-28 02:23:21 +00:00
|
|
|
format_settings.avro.allow_missing_fields = settings.input_format_avro_allow_missing_fields;
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2019-02-19 18:41:18 +00:00
|
|
|
return format_settings;
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
static FormatSettings getOutputFormatSetting(const Settings & settings, const Context & context)
|
2018-06-10 19:22:49 +00:00
|
|
|
{
|
|
|
|
FormatSettings format_settings;
|
2020-04-26 13:44:11 +00:00
|
|
|
format_settings.enable_streaming = settings.output_format_enable_streaming;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
|
|
|
|
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
|
2018-08-06 14:11:45 +00:00
|
|
|
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.csv.delimiter = settings.format_csv_delimiter;
|
2018-07-04 21:00:50 +00:00
|
|
|
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
|
|
|
|
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
|
2020-01-31 13:02:53 +00:00
|
|
|
format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
|
2018-08-30 23:34:12 +00:00
|
|
|
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
|
2020-05-31 22:12:13 +00:00
|
|
|
format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.pretty.color = settings.output_format_pretty_color;
|
2020-07-09 21:02:37 +00:00
|
|
|
format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ?
|
|
|
|
FormatSettings::Pretty::Charset::ASCII :
|
|
|
|
FormatSettings::Pretty::Charset::UTF8;
|
2020-09-29 12:30:36 +00:00
|
|
|
format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers;
|
2019-09-24 14:25:22 +00:00
|
|
|
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;
|
2020-02-03 07:40:12 +00:00
|
|
|
format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line;
|
2020-09-08 12:37:18 +00:00
|
|
|
format_settings.tsv.null_representation = settings.output_format_tsv_null_representation;
|
2018-06-10 19:22:49 +00:00
|
|
|
format_settings.write_statistics = settings.output_format_write_statistics;
|
2019-02-19 20:51:44 +00:00
|
|
|
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
|
2019-12-25 19:17:41 +00:00
|
|
|
format_settings.schema.format_schema = settings.format_schema;
|
|
|
|
format_settings.schema.format_schema_path = context.getFormatSchemaPath();
|
|
|
|
format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER);
|
|
|
|
format_settings.custom.result_before_delimiter = settings.format_custom_result_before_delimiter;
|
|
|
|
format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter;
|
|
|
|
format_settings.custom.escaping_rule = settings.format_custom_escaping_rule;
|
|
|
|
format_settings.custom.field_delimiter = settings.format_custom_field_delimiter;
|
|
|
|
format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter;
|
|
|
|
format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter;
|
|
|
|
format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter;
|
2020-01-11 07:01:20 +00:00
|
|
|
format_settings.avro.output_codec = settings.output_format_avro_codec;
|
|
|
|
format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval;
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2019-02-19 18:41:18 +00:00
|
|
|
return format_settings;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-07-08 13:00:54 +00:00
|
|
|
BlockInputStreamPtr FormatFactory::getInput(
|
|
|
|
const String & name,
|
|
|
|
ReadBuffer & buf,
|
|
|
|
const Block & sample,
|
|
|
|
const Context & context,
|
|
|
|
UInt64 max_block_size,
|
|
|
|
ReadCallback callback) const
|
2019-02-19 18:41:18 +00:00
|
|
|
{
|
2019-07-31 13:26:08 +00:00
|
|
|
if (name == "Native")
|
|
|
|
return std::make_shared<NativeBlockInputStream>(buf, sample, 0);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-08-02 14:41:19 +00:00
|
|
|
if (!getCreators(name).input_processor_creator)
|
2019-08-14 15:54:51 +00:00
|
|
|
{
|
2019-10-01 10:48:46 +00:00
|
|
|
const auto & input_getter = getCreators(name).input_creator;
|
2019-08-14 15:54:51 +00:00
|
|
|
if (!input_getter)
|
|
|
|
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-08-14 15:54:51 +00:00
|
|
|
const Settings & settings = context.getSettingsRef();
|
2019-12-25 19:17:41 +00:00
|
|
|
FormatSettings format_settings = getInputFormatSetting(settings, context);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
return input_getter(buf, sample, max_block_size, callback ? callback : ReadCallback(), format_settings);
|
2019-08-14 15:54:51 +00:00
|
|
|
}
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-10-01 10:48:46 +00:00
|
|
|
const Settings & settings = context.getSettingsRef();
|
|
|
|
const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine;
|
|
|
|
|
2019-11-19 13:11:04 +00:00
|
|
|
// Doesn't make sense to use parallel parsing with less than four threads
|
|
|
|
// (segmentator + two parsers + reader).
|
2020-02-07 13:16:51 +00:00
|
|
|
bool parallel_parsing = settings.input_format_parallel_parsing && file_segmentation_engine && settings.max_threads >= 4;
|
|
|
|
|
|
|
|
if (parallel_parsing && name == "JSONEachRow")
|
|
|
|
{
|
|
|
|
/// FIXME ParallelParsingBlockInputStream doesn't support formats with non-trivial readPrefix() and readSuffix()
|
|
|
|
|
|
|
|
/// For JSONEachRow we can safely skip whitespace characters
|
|
|
|
skipWhitespaceIfAny(buf);
|
|
|
|
if (buf.eof() || *buf.position() == '[')
|
|
|
|
parallel_parsing = false; /// Disable it for JSONEachRow if data is in square brackets (see JSONEachRowRowInputFormat)
|
|
|
|
}
|
|
|
|
|
|
|
|
if (parallel_parsing)
|
2019-10-01 10:48:46 +00:00
|
|
|
{
|
|
|
|
const auto & input_getter = getCreators(name).input_processor_creator;
|
|
|
|
if (!input_getter)
|
|
|
|
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
|
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
FormatSettings format_settings = getInputFormatSetting(settings, context);
|
2019-10-01 10:48:46 +00:00
|
|
|
|
|
|
|
RowInputFormatParams row_input_format_params;
|
|
|
|
row_input_format_params.max_block_size = max_block_size;
|
|
|
|
row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num;
|
|
|
|
row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio;
|
|
|
|
row_input_format_params.callback = std::move(callback);
|
|
|
|
row_input_format_params.max_execution_time = settings.max_execution_time;
|
|
|
|
row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode;
|
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, row_input_format_params, format_settings};
|
2019-11-19 13:11:04 +00:00
|
|
|
ParallelParsingBlockInputStream::Params params{buf, input_getter,
|
|
|
|
input_creator_params, file_segmentation_engine,
|
|
|
|
static_cast<int>(settings.max_threads),
|
|
|
|
settings.min_chunk_bytes_for_parallel_parsing};
|
|
|
|
return std::make_shared<ParallelParsingBlockInputStream>(params);
|
2019-10-01 10:48:46 +00:00
|
|
|
}
|
|
|
|
|
2019-09-20 12:12:32 +00:00
|
|
|
auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback));
|
2019-07-24 18:00:09 +00:00
|
|
|
return std::make_shared<InputStreamFromInputFormat>(std::move(format));
|
2019-02-19 18:41:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-20 11:17:57 +00:00
|
|
|
BlockOutputStreamPtr FormatFactory::getOutput(
|
|
|
|
const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const
|
2019-02-19 18:41:18 +00:00
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
if (!getCreators(name).output_processor_creator)
|
2019-08-14 15:54:51 +00:00
|
|
|
{
|
|
|
|
const auto & output_getter = getCreators(name).output_creator;
|
|
|
|
if (!output_getter)
|
|
|
|
throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
|
|
|
|
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2019-12-25 19:17:41 +00:00
|
|
|
FormatSettings format_settings = getOutputFormatSetting(settings, context);
|
2019-08-14 15:54:51 +00:00
|
|
|
|
2019-08-20 11:17:57 +00:00
|
|
|
/** Materialization is needed, because formats can use the functions `IDataType`,
|
2019-08-14 15:54:51 +00:00
|
|
|
* which only work with full columns.
|
|
|
|
*/
|
|
|
|
return std::make_shared<MaterializingBlockOutputStream>(
|
2019-12-25 19:17:41 +00:00
|
|
|
output_getter(buf, sample, std::move(callback), format_settings), sample);
|
2019-08-14 15:54:51 +00:00
|
|
|
}
|
2019-07-31 10:28:54 +00:00
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
auto format = getOutputFormat(name, buf, sample, context, std::move(callback));
|
2019-07-25 08:37:21 +00:00
|
|
|
return std::make_shared<MaterializingBlockOutputStream>(std::make_shared<OutputStreamToOutputFormat>(format), sample);
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-07-24 18:00:09 +00:00
|
|
|
InputFormatPtr FormatFactory::getInputFormat(
|
|
|
|
const String & name,
|
|
|
|
ReadBuffer & buf,
|
|
|
|
const Block & sample,
|
|
|
|
const Context & context,
|
|
|
|
UInt64 max_block_size,
|
|
|
|
ReadCallback callback) const
|
2019-02-19 18:41:18 +00:00
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
const auto & input_getter = getCreators(name).input_processor_creator;
|
2019-02-19 18:41:18 +00:00
|
|
|
if (!input_getter)
|
|
|
|
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
|
|
|
|
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2019-12-25 19:17:41 +00:00
|
|
|
FormatSettings format_settings = getInputFormatSetting(settings, context);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
|
|
|
RowInputFormatParams params;
|
|
|
|
params.max_block_size = max_block_size;
|
|
|
|
params.allow_errors_num = format_settings.input_allow_errors_num;
|
|
|
|
params.allow_errors_ratio = format_settings.input_allow_errors_ratio;
|
2019-07-24 18:00:09 +00:00
|
|
|
params.callback = std::move(callback);
|
2019-08-01 14:25:41 +00:00
|
|
|
params.max_execution_time = settings.max_execution_time;
|
|
|
|
params.timeout_overflow_mode = settings.timeout_overflow_mode;
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
auto format = input_getter(buf, sample, params, format_settings);
|
|
|
|
|
|
|
|
/// It's a kludge. Because I cannot remove context from values format.
|
|
|
|
if (auto * values = typeid_cast<ValuesBlockInputFormat *>(format.get()))
|
|
|
|
values->setContext(context);
|
|
|
|
|
|
|
|
return format;
|
2019-02-19 18:41:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-20 11:17:57 +00:00
|
|
|
OutputFormatPtr FormatFactory::getOutputFormat(
|
|
|
|
const String & name, WriteBuffer & buf, const Block & sample, const Context & context, WriteCallback callback) const
|
2019-02-19 18:41:18 +00:00
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
const auto & output_getter = getCreators(name).output_processor_creator;
|
2019-02-19 18:41:18 +00:00
|
|
|
if (!output_getter)
|
|
|
|
throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
|
|
|
|
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2019-12-25 19:17:41 +00:00
|
|
|
FormatSettings format_settings = getOutputFormatSetting(settings, context);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
|
|
|
/** TODO: Materialization is needed, because formats can use the functions `IDataType`,
|
|
|
|
* which only work with full columns.
|
|
|
|
*/
|
2019-12-25 19:17:41 +00:00
|
|
|
auto format = output_getter(buf, sample, std::move(callback), format_settings);
|
|
|
|
|
2020-04-27 15:00:10 +00:00
|
|
|
/// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query.
|
|
|
|
if (format_settings.enable_streaming)
|
|
|
|
format->setAutoFlush();
|
|
|
|
|
2019-12-25 19:17:41 +00:00
|
|
|
/// It's a kludge. Because I cannot remove context from MySQL format.
|
|
|
|
if (auto * mysql = typeid_cast<MySQLOutputFormat *>(format.get()))
|
|
|
|
mysql->setContext(context);
|
|
|
|
|
|
|
|
return format;
|
2019-02-19 18:41:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
|
|
|
|
{
|
2019-10-01 10:48:46 +00:00
|
|
|
auto & target = dict[name].input_creator;
|
2018-06-10 19:22:49 +00:00
|
|
|
if (target)
|
|
|
|
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
2019-02-19 18:41:18 +00:00
|
|
|
target = std::move(input_creator);
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator)
|
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
auto & target = dict[name].output_creator;
|
2018-06-10 19:22:49 +00:00
|
|
|
if (target)
|
|
|
|
throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
2019-02-19 18:41:18 +00:00
|
|
|
target = std::move(output_creator);
|
|
|
|
}
|
|
|
|
|
|
|
|
void FormatFactory::registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator)
|
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
auto & target = dict[name].input_processor_creator;
|
2019-02-19 18:41:18 +00:00
|
|
|
if (target)
|
|
|
|
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
target = std::move(input_creator);
|
|
|
|
}
|
|
|
|
|
|
|
|
void FormatFactory::registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator)
|
|
|
|
{
|
2019-08-02 14:41:19 +00:00
|
|
|
auto & target = dict[name].output_processor_creator;
|
2019-02-19 18:41:18 +00:00
|
|
|
if (target)
|
|
|
|
throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
target = std::move(output_creator);
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
2019-10-01 10:48:46 +00:00
|
|
|
void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine)
|
|
|
|
{
|
|
|
|
auto & target = dict[name].file_segmentation_engine;
|
|
|
|
if (target)
|
|
|
|
throw Exception("FormatFactory: File segmentation engine " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
2019-12-25 19:17:41 +00:00
|
|
|
target = std::move(file_segmentation_engine);
|
2019-10-01 10:48:46 +00:00
|
|
|
}
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2020-09-01 08:36:27 +00:00
|
|
|
/// File Segmentation Engines for parallel reading
|
|
|
|
|
|
|
|
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
|
|
|
|
void registerFileSegmentationEngineCSV(FormatFactory & factory);
|
|
|
|
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
|
|
|
|
void registerFileSegmentationEngineRegexp(FormatFactory & factory);
|
|
|
|
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory);
|
2020-09-15 10:13:39 +00:00
|
|
|
void registerFileSegmentationEngineLineAsString(FormatFactory & factory);
|
2020-09-01 08:36:27 +00:00
|
|
|
|
|
|
|
/// Formats for both input/output.
|
|
|
|
|
|
|
|
void registerInputFormatNative(FormatFactory & factory);
|
|
|
|
void registerOutputFormatNative(FormatFactory & factory);
|
|
|
|
|
|
|
|
void registerInputFormatProcessorNative(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorNative(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorRowBinary(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorRowBinary(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorTabSeparated(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorTabSeparated(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorValues(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorValues(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorCSV(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorCSV(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorTSKV(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorTSKV(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorJSONEachRow(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorProtobuf(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorTemplate(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorTemplate(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorMsgPack(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorMsgPack(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorORC(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorORC(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorParquet(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorParquet(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorArrow(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorArrow(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorAvro(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorAvro(FormatFactory & factory);
|
2020-09-28 00:11:19 +00:00
|
|
|
void registerInputFormatProcessorRawBLOB(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorRawBLOB(FormatFactory & factory);
|
2020-09-01 08:36:27 +00:00
|
|
|
|
|
|
|
/// Output only (presentational) formats.
|
|
|
|
|
|
|
|
void registerOutputFormatNull(FormatFactory & factory);
|
|
|
|
|
|
|
|
void registerOutputFormatProcessorPretty(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorPrettySpace(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorVertical(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorJSON(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorJSONCompact(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorXML(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorNull(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorMySQLWire(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorMarkdown(FormatFactory & factory);
|
|
|
|
void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory);
|
|
|
|
|
|
|
|
/// Input only formats.
|
|
|
|
|
|
|
|
void registerInputFormatProcessorRegexp(FormatFactory & factory);
|
|
|
|
void registerInputFormatProcessorJSONAsString(FormatFactory & factory);
|
2020-09-10 14:10:47 +00:00
|
|
|
void registerInputFormatProcessorLineAsString(FormatFactory & factory);
|
2020-09-01 08:36:27 +00:00
|
|
|
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
FormatFactory::FormatFactory()
|
|
|
|
{
|
2020-09-01 08:36:27 +00:00
|
|
|
registerFileSegmentationEngineTabSeparated(*this);
|
|
|
|
registerFileSegmentationEngineCSV(*this);
|
|
|
|
registerFileSegmentationEngineJSONEachRow(*this);
|
|
|
|
registerFileSegmentationEngineRegexp(*this);
|
|
|
|
registerFileSegmentationEngineJSONAsString(*this);
|
2020-09-15 10:13:39 +00:00
|
|
|
registerFileSegmentationEngineLineAsString(*this);
|
2020-09-01 08:36:27 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
registerInputFormatNative(*this);
|
|
|
|
registerOutputFormatNative(*this);
|
|
|
|
|
2019-02-19 18:41:18 +00:00
|
|
|
registerInputFormatProcessorNative(*this);
|
|
|
|
registerOutputFormatProcessorNative(*this);
|
|
|
|
registerInputFormatProcessorRowBinary(*this);
|
|
|
|
registerOutputFormatProcessorRowBinary(*this);
|
|
|
|
registerInputFormatProcessorTabSeparated(*this);
|
|
|
|
registerOutputFormatProcessorTabSeparated(*this);
|
|
|
|
registerInputFormatProcessorValues(*this);
|
|
|
|
registerOutputFormatProcessorValues(*this);
|
|
|
|
registerInputFormatProcessorCSV(*this);
|
|
|
|
registerOutputFormatProcessorCSV(*this);
|
|
|
|
registerInputFormatProcessorTSKV(*this);
|
|
|
|
registerOutputFormatProcessorTSKV(*this);
|
|
|
|
registerInputFormatProcessorJSONEachRow(*this);
|
|
|
|
registerOutputFormatProcessorJSONEachRow(*this);
|
2019-12-05 10:13:40 +00:00
|
|
|
registerInputFormatProcessorJSONCompactEachRow(*this);
|
|
|
|
registerOutputFormatProcessorJSONCompactEachRow(*this);
|
2019-02-21 18:36:46 +00:00
|
|
|
registerInputFormatProcessorProtobuf(*this);
|
2019-02-19 18:41:18 +00:00
|
|
|
registerOutputFormatProcessorProtobuf(*this);
|
2020-09-01 08:36:27 +00:00
|
|
|
registerInputFormatProcessorTemplate(*this);
|
|
|
|
registerOutputFormatProcessorTemplate(*this);
|
|
|
|
registerInputFormatProcessorMsgPack(*this);
|
|
|
|
registerOutputFormatProcessorMsgPack(*this);
|
2020-09-28 00:11:19 +00:00
|
|
|
registerInputFormatProcessorRawBLOB(*this);
|
|
|
|
registerOutputFormatProcessorRawBLOB(*this);
|
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
2019-08-21 14:19:47 +00:00
|
|
|
registerInputFormatProcessorORC(*this);
|
2020-06-30 11:38:09 +00:00
|
|
|
registerOutputFormatProcessorORC(*this);
|
2019-02-21 18:36:46 +00:00
|
|
|
registerInputFormatProcessorParquet(*this);
|
|
|
|
registerOutputFormatProcessorParquet(*this);
|
2020-04-28 19:52:22 +00:00
|
|
|
registerInputFormatProcessorArrow(*this);
|
2020-05-03 02:46:36 +00:00
|
|
|
registerOutputFormatProcessorArrow(*this);
|
2020-01-08 09:13:12 +00:00
|
|
|
registerInputFormatProcessorAvro(*this);
|
|
|
|
registerOutputFormatProcessorAvro(*this);
|
2020-04-16 12:31:57 +00:00
|
|
|
#endif
|
2019-08-02 16:20:25 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
registerOutputFormatNull(*this);
|
2019-02-19 18:41:18 +00:00
|
|
|
|
|
|
|
registerOutputFormatProcessorPretty(*this);
|
|
|
|
registerOutputFormatProcessorPrettyCompact(*this);
|
|
|
|
registerOutputFormatProcessorPrettySpace(*this);
|
|
|
|
registerOutputFormatProcessorVertical(*this);
|
|
|
|
registerOutputFormatProcessorJSON(*this);
|
|
|
|
registerOutputFormatProcessorJSONCompact(*this);
|
2020-09-01 08:36:27 +00:00
|
|
|
registerOutputFormatProcessorJSONEachRowWithProgress(*this);
|
2019-02-19 18:41:18 +00:00
|
|
|
registerOutputFormatProcessorXML(*this);
|
|
|
|
registerOutputFormatProcessorODBCDriver2(*this);
|
|
|
|
registerOutputFormatProcessorNull(*this);
|
2020-06-21 12:16:08 +00:00
|
|
|
registerOutputFormatProcessorMySQLWire(*this);
|
2020-04-16 20:10:42 +00:00
|
|
|
registerOutputFormatProcessorMarkdown(*this);
|
2020-06-21 12:16:08 +00:00
|
|
|
registerOutputFormatProcessorPostgreSQLWire(*this);
|
2020-09-01 08:36:27 +00:00
|
|
|
|
|
|
|
registerInputFormatProcessorRegexp(*this);
|
|
|
|
registerInputFormatProcessorJSONAsString(*this);
|
2020-09-10 14:10:47 +00:00
|
|
|
registerInputFormatProcessorLineAsString(*this);
|
2020-09-28 00:11:19 +00:00
|
|
|
|
2020-09-01 08:36:27 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
registerInputFormatProcessorCapnProto(*this);
|
|
|
|
#endif
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
2019-08-22 03:24:05 +00:00
|
|
|
FormatFactory & FormatFactory::instance()
|
|
|
|
{
|
|
|
|
static FormatFactory ret;
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
}
|