Fix inconsistency in FormatFactory

This commit is contained in:
Alexey Milovidov 2020-10-29 06:39:43 +03:00
parent d46cf39f3b
commit 2e0a979e3a
10 changed files with 182 additions and 147 deletions

View File

@ -75,6 +75,7 @@
#include <Common/InterruptListener.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Formats/registerFormats.h>
#include <Common/Config/configReadClient.h>
#include <Storages/ColumnsDescription.h>
#include <common/argsToConfig.h>
@ -461,6 +462,7 @@ private:
{
UseSSL use_ssl;
registerFormats();
registerFunctions();
registerAggregateFunctions();

View File

@ -1,6 +1,7 @@
#include "ClusterCopierApp.h"
#include <Common/StatusFile.h>
#include <Common/TerminalSize.h>
#include <Formats/registerFormats.h>
#include <unistd.h>
@ -122,6 +123,7 @@ void ClusterCopierApp::mainImpl()
registerStorages();
registerDictionaries();
registerDisks();
registerFormats();
static const std::string default_database = "_local";
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, *context));

View File

@ -33,6 +33,7 @@
#include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
#include <boost/program_options/options_description.hpp>
#include <boost/program_options.hpp>
#include <common/argsToConfig.h>
@ -224,6 +225,7 @@ try
registerStorages();
registerDictionaries();
registerDisks();
registerFormats();
/// Maybe useless
if (config().has("macros"))

View File

@ -18,11 +18,13 @@
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/config.h>
#include <Formats/registerFormats.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/range.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
namespace ErrorCodes
@ -160,6 +162,8 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
if (is_help)
return Application::EXIT_OK;
registerFormats();
LOG_INFO(log, "Starting up");
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, hostname, port, log);

View File

@ -51,6 +51,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Formats/registerFormats.h>
#include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
@ -266,6 +267,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
registerStorages();
registerDictionaries();
registerDisks();
registerFormats();
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());

View File

@ -333,150 +333,6 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm
target = std::move(file_segmentation_engine);
}
/// 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);
void registerFileSegmentationEngineLineAsString(FormatFactory & factory);
/// 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);
void registerInputFormatProcessorRawBLOB(FormatFactory & factory);
void registerOutputFormatProcessorRawBLOB(FormatFactory & factory);
/// 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);
void registerInputFormatProcessorLineAsString(FormatFactory & factory);
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
FormatFactory::FormatFactory()
{
registerFileSegmentationEngineTabSeparated(*this);
registerFileSegmentationEngineCSV(*this);
registerFileSegmentationEngineJSONEachRow(*this);
registerFileSegmentationEngineRegexp(*this);
registerFileSegmentationEngineJSONAsString(*this);
registerFileSegmentationEngineLineAsString(*this);
registerInputFormatNative(*this);
registerOutputFormatNative(*this);
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);
registerInputFormatProcessorJSONCompactEachRow(*this);
registerOutputFormatProcessorJSONCompactEachRow(*this);
registerInputFormatProcessorProtobuf(*this);
registerOutputFormatProcessorProtobuf(*this);
registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this);
registerInputFormatProcessorMsgPack(*this);
registerOutputFormatProcessorMsgPack(*this);
registerInputFormatProcessorRawBLOB(*this);
registerOutputFormatProcessorRawBLOB(*this);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorORC(*this);
registerOutputFormatProcessorORC(*this);
registerInputFormatProcessorParquet(*this);
registerOutputFormatProcessorParquet(*this);
registerInputFormatProcessorArrow(*this);
registerOutputFormatProcessorArrow(*this);
registerInputFormatProcessorAvro(*this);
registerOutputFormatProcessorAvro(*this);
#endif
registerOutputFormatNull(*this);
registerOutputFormatProcessorPretty(*this);
registerOutputFormatProcessorPrettyCompact(*this);
registerOutputFormatProcessorPrettySpace(*this);
registerOutputFormatProcessorVertical(*this);
registerOutputFormatProcessorJSON(*this);
registerOutputFormatProcessorJSONCompact(*this);
registerOutputFormatProcessorJSONEachRowWithProgress(*this);
registerOutputFormatProcessorXML(*this);
registerOutputFormatProcessorODBCDriver2(*this);
registerOutputFormatProcessorNull(*this);
registerOutputFormatProcessorMySQLWire(*this);
registerOutputFormatProcessorMarkdown(*this);
registerOutputFormatProcessorPostgreSQLWire(*this);
registerInputFormatProcessorRegexp(*this);
registerInputFormatProcessorJSONAsString(*this);
registerInputFormatProcessorLineAsString(*this);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorCapnProto(*this);
#endif
}
FormatFactory & FormatFactory::instance()
{

View File

@ -96,7 +96,6 @@ private:
using FormatsDictionary = std::unordered_map<String, Creators>;
public:
static FormatFactory & instance();
BlockInputStreamPtr getInput(
@ -137,8 +136,6 @@ public:
private:
FormatsDictionary dict;
FormatFactory();
const Creators & getCreators(const String & name) const;
};

View File

@ -0,0 +1,160 @@
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
#include <Formats/FormatFactory.h>
namespace DB
{
/// 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);
void registerFileSegmentationEngineLineAsString(FormatFactory & factory);
/// 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);
void registerInputFormatProcessorRawBLOB(FormatFactory & factory);
void registerOutputFormatProcessorRawBLOB(FormatFactory & factory);
/// 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);
void registerInputFormatProcessorLineAsString(FormatFactory & factory);
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
void registerFormats()
{
auto & factory = FormatFactory::instance();
registerFileSegmentationEngineTabSeparated(factory);
registerFileSegmentationEngineCSV(factory);
registerFileSegmentationEngineJSONEachRow(factory);
registerFileSegmentationEngineRegexp(factory);
registerFileSegmentationEngineJSONAsString(factory);
registerFileSegmentationEngineLineAsString(factory);
registerInputFormatNative(factory);
registerOutputFormatNative(factory);
registerInputFormatProcessorNative(factory);
registerOutputFormatProcessorNative(factory);
registerInputFormatProcessorRowBinary(factory);
registerOutputFormatProcessorRowBinary(factory);
registerInputFormatProcessorTabSeparated(factory);
registerOutputFormatProcessorTabSeparated(factory);
registerInputFormatProcessorValues(factory);
registerOutputFormatProcessorValues(factory);
registerInputFormatProcessorCSV(factory);
registerOutputFormatProcessorCSV(factory);
registerInputFormatProcessorTSKV(factory);
registerOutputFormatProcessorTSKV(factory);
registerInputFormatProcessorJSONEachRow(factory);
registerOutputFormatProcessorJSONEachRow(factory);
registerInputFormatProcessorJSONCompactEachRow(factory);
registerOutputFormatProcessorJSONCompactEachRow(factory);
registerInputFormatProcessorProtobuf(factory);
registerOutputFormatProcessorProtobuf(factory);
registerInputFormatProcessorTemplate(factory);
registerOutputFormatProcessorTemplate(factory);
registerInputFormatProcessorMsgPack(factory);
registerOutputFormatProcessorMsgPack(factory);
registerInputFormatProcessorRawBLOB(factory);
registerOutputFormatProcessorRawBLOB(factory);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorORC(factory);
registerOutputFormatProcessorORC(factory);
registerInputFormatProcessorParquet(factory);
registerOutputFormatProcessorParquet(factory);
registerInputFormatProcessorArrow(factory);
registerOutputFormatProcessorArrow(factory);
registerInputFormatProcessorAvro(factory);
registerOutputFormatProcessorAvro(factory);
#endif
registerOutputFormatNull(factory);
registerOutputFormatProcessorPretty(factory);
registerOutputFormatProcessorPrettyCompact(factory);
registerOutputFormatProcessorPrettySpace(factory);
registerOutputFormatProcessorVertical(factory);
registerOutputFormatProcessorJSON(factory);
registerOutputFormatProcessorJSONCompact(factory);
registerOutputFormatProcessorJSONEachRowWithProgress(factory);
registerOutputFormatProcessorXML(factory);
registerOutputFormatProcessorODBCDriver2(factory);
registerOutputFormatProcessorNull(factory);
registerOutputFormatProcessorMySQLWire(factory);
registerOutputFormatProcessorMarkdown(factory);
registerOutputFormatProcessorPostgreSQLWire(factory);
registerInputFormatProcessorRegexp(factory);
registerInputFormatProcessorJSONAsString(factory);
registerInputFormatProcessorLineAsString(factory);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorCapnProto(factory);
#endif
}
}

View File

@ -0,0 +1,9 @@
#pragma once
namespace DB
{
void registerFormats();
}

View File

@ -23,6 +23,7 @@ SRCS(
ProtobufReader.cpp
ProtobufSchemas.cpp
ProtobufWriter.cpp
registerFormats.cpp
verbosePrintString.cpp
)