mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix inconsistency in FormatFactory
This commit is contained in:
parent
d46cf39f3b
commit
2e0a979e3a
@ -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();
|
||||
|
||||
|
@ -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));
|
||||
|
@ -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"))
|
||||
|
@ -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);
|
||||
|
@ -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());
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
160
src/Formats/registerFormats.cpp
Normal file
160
src/Formats/registerFormats.cpp
Normal 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
|
||||
}
|
||||
|
||||
}
|
||||
|
9
src/Formats/registerFormats.h
Normal file
9
src/Formats/registerFormats.h
Normal file
@ -0,0 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFormats();
|
||||
|
||||
}
|
||||
|
@ -23,6 +23,7 @@ SRCS(
|
||||
ProtobufReader.cpp
|
||||
ProtobufSchemas.cpp
|
||||
ProtobufWriter.cpp
|
||||
registerFormats.cpp
|
||||
verbosePrintString.cpp
|
||||
|
||||
)
|
||||
|
Loading…
Reference in New Issue
Block a user