Merge pull request #16496 from ClickHouse/register-formats

Fix inconsistency in FormatFactory
This commit is contained in:
alexey-milovidov 2020-11-08 14:27:33 +03:00 committed by GitHub
commit 2d427eb1e8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 235 additions and 198 deletions

View File

@ -227,9 +227,6 @@ else ()
install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import)
endif ()
if(ENABLE_CLICKHOUSE_ODBC_BRIDGE)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge)
endif()
install (TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)

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>
@ -463,6 +464,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

@ -23,6 +23,7 @@
#include <Common/HashTable/HashMap.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Formats/registerFormats.h>
#include <Core/Block.h>
#include <common/StringRef.h>
#include <common/DateLUT.h>
@ -1050,6 +1051,8 @@ try
using namespace DB;
namespace po = boost::program_options;
registerFormats();
po::options_description description = createOptionsDescription("Options", getTerminalWidth());
description.add_options()
("help", "produce help message")

View File

@ -10,19 +10,8 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES
PingHandler.cpp
SchemaAllowedHandler.cpp
validateODBCConnectionString.cpp
odbc-bridge.cpp
)
set (CLICKHOUSE_ODBC_BRIDGE_LINK
PRIVATE
clickhouse_parsers
clickhouse_aggregate_functions
daemon
dbms
Poco::Data
PUBLIC
Poco::Data::ODBC
)
clickhouse_program_add_library(odbc-bridge)
if (OS_LINUX)
# clickhouse-odbc-bridge is always a separate binary.
@ -30,10 +19,17 @@ if (OS_LINUX)
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic")
endif ()
add_executable(clickhouse-odbc-bridge odbc-bridge.cpp)
set_target_properties(clickhouse-odbc-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES})
clickhouse_program_link_split_binary(odbc-bridge)
target_link_libraries(clickhouse-odbc-bridge PRIVATE
daemon
dbms
clickhouse_parsers
Poco::Data
Poco::Data::ODBC
)
set_target_properties(clickhouse-odbc-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
if (USE_GDB_ADD_INDEX)
add_custom_command(TARGET clickhouse-odbc-bridge POST_BUILD COMMAND ${GDB_ADD_INDEX_EXE} ../clickhouse-odbc-bridge COMMENT "Adding .gdb-index to clickhouse-odbc-bridge" VERBATIM)

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

@ -1,3 +1,2 @@
add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp)
clickhouse_target_link_split_lib(validate-odbc-connection-string odbc-bridge)
add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp ../validateODBCConnectionString.cpp)
target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse_common_io)

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

@ -0,0 +1,15 @@
#pragma once
#include <Functions/registerFunctions.h>
#include <Formats/registerFormats.h>
inline void tryRegisterFunctions()
{
static struct Register { Register() { DB::registerFunctions(); } } registered;
}
inline void tryRegisterFormats()
{
static struct Register { Register() { DB::registerFormats(); } } registered;
}

View File

@ -1,18 +0,0 @@
#pragma once
#include <Functions/FunctionFactory.h>
#include <Functions/registerFunctions.h>
struct RegisteredFunctionsState
{
RegisteredFunctionsState()
{
DB::registerFunctions();
}
RegisteredFunctionsState(RegisteredFunctionsState &&) = default;
};
inline void tryRegisterFunctions()
{
static RegisteredFunctionsState registered_functions_state;
}

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

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

View File

@ -10,39 +10,44 @@ Variants test_bayesab(std::string dist, PODArray<Float64> xs, PODArray<Float64>
{
Variants variants;
std::cout << std::fixed;
//std::cout << std::fixed;
if (dist == "beta")
{
std::cout << dist << "\nclicks: ";
for (auto x : xs) std::cout << x << " ";
/* std::cout << dist << "\nclicks: ";
for (auto x : xs)
std::cout << x << " ";
std::cout <<"\tconversions: ";
for (auto y : ys) std::cout << y << " ";
for (auto y : ys)
std::cout << y << " ";
std::cout << "\n";
std::cout << "\n";*/
variants = bayesian_ab_test<true>(dist, xs, ys);
}
else if (dist == "gamma")
{
std::cout << dist << "\nclicks: ";
for (auto x : xs) std::cout << x << " ";
/* std::cout << dist << "\nclicks: ";
for (auto x : xs)
std::cout << x << " ";
std::cout <<"\tcost: ";
for (auto y : ys) std::cout << y << " ";
for (auto y : ys)
std::cout << y << " ";
std::cout << "\n";*/
std::cout << "\n";
variants = bayesian_ab_test<true>(dist, xs, ys);
}
for (size_t i = 0; i < variants.size(); ++i)
/* for (size_t i = 0; i < variants.size(); ++i)
std::cout << i << " beats 0: " << variants[i].beats_control << std::endl;
for (size_t i = 0; i < variants.size(); ++i)
std::cout << i << " to be best: " << variants[i].best << std::endl;
std::cout << convertToJson({"0", "1", "2"}, variants) << std::endl;
*/
Float64 max_val = 0.0, min_val = 2.0;
for (size_t i = 0; i < variants.size(); ++i)
{

View File

@ -12,7 +12,9 @@
#include <Interpreters/Context.h>
#include <Interpreters/MySQL/InterpretersMySQLDDLQuery.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_global_register_functions.h>
#include <Common/tests/gtest_global_register.h>
#include <Poco/String.h>
using namespace DB;

View File

@ -13,6 +13,7 @@
#include <Storages/SelectQueryInfo.h>
#include <Common/typeid_cast.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_global_register.h>
#include <memory>
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
@ -130,6 +131,8 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context)
sample.insert(std::move(col));
}
tryRegisterFormats();
std::ostringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOStream out_buf(ss);

View File

@ -10,7 +10,7 @@
#include <Databases/DatabaseMemory.h>
#include <Storages/StorageMemory.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_global_register_functions.h>
#include <Common/tests/gtest_global_register.h>
using namespace DB;