mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Reduce header dependencies
This commit is contained in:
parent
56e48c5d64
commit
de855ca917
@ -1,8 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/extended_types.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <base/strong_typedef.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
|
||||
namespace DB
|
||||
@ -62,7 +61,8 @@ namespace std
|
||||
{
|
||||
size_t operator()(const DB::IPv6 & x) const
|
||||
{
|
||||
return std::hash<std::string_view>{}(std::string_view(reinterpret_cast<const char*>(&x.toUnderType()), IPV6_BINARY_LENGTH));
|
||||
return std::hash<std::string_view>{}(
|
||||
std::string_view(reinterpret_cast<const char *>(&x.toUnderType()), sizeof(DB::IPv6::UnderlyingType)));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Common/StudentTTest.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
|
||||
|
||||
/** A tool for evaluating ClickHouse performance.
|
||||
@ -623,7 +624,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
;
|
||||
|
||||
Settings settings;
|
||||
settings.addProgramOptions(desc);
|
||||
addProgramOptions(settings, desc);
|
||||
|
||||
boost::program_options::variables_map options;
|
||||
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
@ -102,7 +103,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
{
|
||||
std::string_view name = field.getName();
|
||||
if (name == "max_parser_depth" || name == "max_query_size")
|
||||
cmd_settings.addProgramOption(desc, name, field);
|
||||
addProgramOption(cmd_settings, desc, name, field);
|
||||
}
|
||||
|
||||
boost::program_options::variables_map options;
|
||||
|
@ -36,6 +36,7 @@
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/getExecutablePath.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/ThreadProfileEvents.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/getMappedArea.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <base/range.h>
|
||||
|
||||
#include <boost/algorithm/string/case_conv.hpp>
|
||||
#include <boost/algorithm/string/classification.hpp>
|
||||
|
@ -13,6 +13,8 @@
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
namespace Poco::Net { class IPAddress; }
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Access/User.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
|
||||
|
@ -1,16 +1,16 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
#include <unordered_set>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <bitset>
|
||||
#include <base/range.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <bitset>
|
||||
#include <unordered_set>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <base/range.h>
|
||||
|
||||
#include <bitset>
|
||||
#include <stack>
|
||||
|
||||
|
@ -48,6 +48,7 @@
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Backups/DDLAdjustingForBackupVisitor.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/BackupIO.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
|
||||
|
@ -1,7 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/BackupIO_Default.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Disks/DiskType.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <Backups/BackupIO_Default.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Disks/DiskType.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <base/safeExit.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
@ -2714,9 +2715,9 @@ private:
|
||||
void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
|
||||
{
|
||||
if (allow_repeated_settings)
|
||||
cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value());
|
||||
addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value());
|
||||
else
|
||||
cmd_settings.addProgramOptions(options_description.main_description.value());
|
||||
addProgramOptions(cmd_settings, options_description.main_description.value());
|
||||
|
||||
if (allow_merge_tree_settings)
|
||||
{
|
||||
@ -2737,9 +2738,9 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description,
|
||||
return;
|
||||
|
||||
if (allow_repeated_settings)
|
||||
cmd_merge_tree_settings.addProgramOptionAsMultitoken(main_options, name, setting);
|
||||
addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting);
|
||||
else
|
||||
cmd_merge_tree_settings.addProgramOption(main_options, name, setting);
|
||||
addProgramOption(cmd_merge_tree_settings, main_options, name, setting);
|
||||
};
|
||||
|
||||
const auto & setting_name = setting.getName();
|
||||
|
@ -1,13 +1,14 @@
|
||||
#include "LocalConnection.h"
|
||||
#include <Core/Protocol.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Processors/Executors/PushingPipelineExecutor.h>
|
||||
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
|
||||
#include <Processors/Executors/PushingPipelineExecutor.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Core/Protocol.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -10,14 +10,15 @@
|
||||
#include <Common/MemoryTrackerSwitcher.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
#include <Poco/Net/HTTPStream.h>
|
||||
#include <Poco/Net/HTTPFixedLengthStream.h>
|
||||
#include <Poco/Net/HTTPChunkedStream.h>
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
#include <Poco/Net/HTTPFixedLengthStream.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/Net/HTTPStream.h>
|
||||
#include <Poco/Timespan.h>
|
||||
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <queue>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
|
@ -12,6 +12,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ISchedulerNode;
|
||||
using SchedulerNodePtr = std::shared_ptr<ISchedulerNode>;
|
||||
|
||||
/*
|
||||
* Instance of derived class holds everything required for resource consumption,
|
||||
* including resources currently registered at `SchedulerRoot`. This is required to avoid
|
||||
|
25
src/Common/Scheduler/ResouceLink.cpp
Normal file
25
src/Common/Scheduler/ResouceLink.cpp
Normal file
@ -0,0 +1,25 @@
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void ResourceLink::adjust(ResourceCost estimated_cost, ResourceCost real_cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->adjustBudget(estimated_cost, real_cost);
|
||||
}
|
||||
|
||||
void ResourceLink::consumed(ResourceCost cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->consumeBudget(cost);
|
||||
}
|
||||
|
||||
void ResourceLink::accumulate(DB::ResourceCost cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->accumulateBudget(cost);
|
||||
}
|
||||
}
|
||||
|
@ -2,9 +2,10 @@
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
||||
|
@ -2,12 +2,10 @@
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ISchedulerQueue;
|
||||
using ResourceCost = Int64;
|
||||
|
||||
/*
|
||||
* Everything required for resource consumption. Connection to a specific resource queue.
|
||||
@ -17,23 +15,11 @@ struct ResourceLink
|
||||
ISchedulerQueue * queue = nullptr;
|
||||
bool operator==(const ResourceLink &) const = default;
|
||||
|
||||
void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->adjustBudget(estimated_cost, real_cost);
|
||||
}
|
||||
void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const;
|
||||
|
||||
void consumed(ResourceCost cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->consumeBudget(cost);
|
||||
}
|
||||
void consumed(ResourceCost cost) const;
|
||||
|
||||
void accumulate(ResourceCost cost) const
|
||||
{
|
||||
if (queue)
|
||||
queue->accumulateBudget(cost);
|
||||
}
|
||||
void accumulate(ResourceCost cost) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <Common/SymbolIndex.h>
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <base/hex.h>
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <optional>
|
||||
@ -11,8 +13,6 @@
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
/**
|
||||
|
||||
ELF object can contain three different places with symbol names and addresses:
|
||||
|
@ -1,12 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Core/LogsLevel.h>
|
||||
#include <IO/Progress.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <base/StringRef.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <base/StringRef.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
|
@ -1,13 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <cstring>
|
||||
#include <algorithm>
|
||||
#include <cstring>
|
||||
#include <type_traits>
|
||||
#include <utility>
|
||||
#include <base/range.h>
|
||||
#include <base/unaligned.h>
|
||||
#include <base/hex.h>
|
||||
#include <base/types.h>
|
||||
#include <base/unaligned.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
constexpr size_t IPV4_BINARY_LENGTH = 4;
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Common/tests/gtest_global_context.h>
|
||||
#include <Common/tests/gtest_helper_functions.h>
|
||||
|
||||
#include <Poco/Util/MapConfiguration.h>
|
||||
|
||||
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
class ProxyConfigurationResolverProviderTests : public ::testing::Test
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <base/range.h>
|
||||
#include <boost/blank.hpp>
|
||||
#include <unordered_map>
|
||||
#include <boost/program_options/options_description.hpp>
|
||||
|
||||
|
||||
namespace boost::program_options
|
||||
@ -129,18 +128,6 @@ public:
|
||||
std::conditional_t<Traits::allow_custom_settings, const CustomSettingMap::mapped_type*, boost::blank> custom_setting;
|
||||
};
|
||||
|
||||
/// Adds program options to set the settings from a command line.
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
void addProgramOptions(boost::program_options::options_description & options);
|
||||
|
||||
/// Adds program options as to set the settings from a command line.
|
||||
/// Allows to set one setting multiple times, the last value will be used.
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
void addProgramOptionsAsMultitokens(boost::program_options::options_description & options);
|
||||
|
||||
void addProgramOption(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field);
|
||||
void addProgramOptionAsMultitoken(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field);
|
||||
|
||||
enum SkipFlags
|
||||
{
|
||||
SKIP_NONE = 0,
|
||||
@ -561,57 +548,6 @@ String BaseSettings<TTraits>::toString() const
|
||||
return out.str();
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
void BaseSettings<TTraits>::addProgramOptions(boost::program_options::options_description & options)
|
||||
{
|
||||
const auto & settings_to_aliases = TTraits::settingsToAliases();
|
||||
for (const auto & field : all())
|
||||
{
|
||||
std::string_view name = field.getName();
|
||||
addProgramOption(options, name, field);
|
||||
|
||||
if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end())
|
||||
{
|
||||
for (const auto alias : it->second)
|
||||
addProgramOption(options, alias, field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
void BaseSettings<TTraits>::addProgramOptionsAsMultitokens(boost::program_options::options_description & options)
|
||||
{
|
||||
const auto & settings_to_aliases = TTraits::settingsToAliases();
|
||||
for (const auto & field : all())
|
||||
{
|
||||
std::string_view name = field.getName();
|
||||
addProgramOptionAsMultitoken(options, name, field);
|
||||
|
||||
if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end())
|
||||
{
|
||||
for (const auto alias : it->second)
|
||||
addProgramOptionAsMultitoken(options, alias, field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename TTraits>
|
||||
void BaseSettings<TTraits>::addProgramOption(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field)
|
||||
{
|
||||
auto on_program_option = boost::function1<void, const std::string &>([this, name](const std::string & value) { set(name, value); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<std::string>()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
void BaseSettings<TTraits>::addProgramOptionAsMultitoken(boost::program_options::options_description & options, std::string_view name, const SettingFieldRef & field)
|
||||
{
|
||||
auto on_program_option = boost::function1<void, const Strings &>([this, name](const Strings & values) { set(name, values.back()); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<Strings>()->multitoken()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
template <typename TTraits>
|
||||
bool operator==(const BaseSettings<TTraits> & left, const BaseSettings<TTraits> & right)
|
||||
{
|
||||
|
60
src/Core/BaseSettingsProgramOptions.h
Normal file
60
src/Core/BaseSettingsProgramOptions.h
Normal file
@ -0,0 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
void addProgramOptionAsMultitoken(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field)
|
||||
{
|
||||
auto on_program_option = boost::function1<void, const Strings &>([&cmd_settings, name](const Strings & values) { cmd_settings.set(name, values.back()); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<Strings>()->multitoken()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void addProgramOptionsAsMultitokens(T &cmd_settings, boost::program_options::options_description & options)
|
||||
{
|
||||
const auto & settings_to_aliases = T::Traits::settingsToAliases();
|
||||
for (const auto & field : cmd_settings.all())
|
||||
{
|
||||
std::string_view name = field.getName();
|
||||
addProgramOptionAsMultitoken(cmd_settings, options, name, field);
|
||||
|
||||
if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end())
|
||||
for (const auto alias : it->second)
|
||||
addProgramOptionAsMultitoken(cmd_settings, options, alias, field);
|
||||
}
|
||||
}
|
||||
|
||||
/// Adds program options to set the settings from a command line.
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
template <typename T>
|
||||
void addProgramOption(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field)
|
||||
{
|
||||
auto on_program_option = boost::function1<void, const std::string &>([&cmd_settings, name](const std::string & value) { cmd_settings.set(name, value); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<std::string>()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void addProgramOptions(T &cmd_settings, boost::program_options::options_description & options)
|
||||
{
|
||||
const auto & settings_to_aliases = T::Traits::settingsToAliases();
|
||||
for (const auto & field : cmd_settings.all())
|
||||
{
|
||||
std::string_view name = field.getName();
|
||||
addProgramOption(cmd_settings, options, name, field);
|
||||
|
||||
if (auto it = settings_to_aliases.find(name); it != settings_to_aliases.end())
|
||||
for (const auto alias : it->second)
|
||||
addProgramOption(cmd_settings, options, alias, field);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
16
src/Core/LogsLevel.h
Normal file
16
src/Core/LogsLevel.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
enum class LogsLevel
|
||||
{
|
||||
none = 0, /// Disable
|
||||
fatal,
|
||||
error,
|
||||
warning,
|
||||
information,
|
||||
debug,
|
||||
trace,
|
||||
test,
|
||||
};
|
||||
}
|
@ -1,12 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Core/Joins.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
#include <Core/LogsLevel.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Common/ShellCommandSettings.h>
|
||||
#include <Parsers/ASTSQLSecurity.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
#include <Common/ShellCommandSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -90,18 +91,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(IntervalOutputFormat, FormatSettings::IntervalO
|
||||
|
||||
DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion)
|
||||
|
||||
enum class LogsLevel
|
||||
{
|
||||
none = 0, /// Disable
|
||||
fatal,
|
||||
error,
|
||||
warning,
|
||||
information,
|
||||
debug,
|
||||
trace,
|
||||
test,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(LogsLevel)
|
||||
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include "config.h"
|
||||
#include <Common/config_version.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <DataTypes/Serializations/SerializationBool.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Serializations/SerializationBool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
|
@ -346,7 +346,7 @@ SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info)
|
||||
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
|
||||
}
|
||||
|
||||
MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const
|
||||
MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfoSettings & settings) const
|
||||
{
|
||||
MutableSerializationInfos infos;
|
||||
infos.reserve(elems.size());
|
||||
|
@ -58,7 +58,7 @@ public:
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
SerializationPtr getSerialization(const SerializationInfo & info) const override;
|
||||
MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override;
|
||||
MutableSerializationInfoPtr createSerializationInfo(const SerializationInfoSettings & settings) const override;
|
||||
SerializationInfoPtr getSerializationInfo(const IColumn & column) const override;
|
||||
|
||||
const DataTypePtr & getElement(size_t i) const { return elems[i]; }
|
||||
|
@ -202,7 +202,7 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const
|
||||
custom_serialization = std::move(custom_desc_->serialization);
|
||||
}
|
||||
|
||||
MutableSerializationInfoPtr IDataType::createSerializationInfo(const SerializationInfo::Settings & settings) const
|
||||
MutableSerializationInfoPtr IDataType::createSerializationInfo(const SerializationInfoSettings & settings) const
|
||||
{
|
||||
return std::make_shared<SerializationInfo>(ISerialization::Kind::DEFAULT, settings);
|
||||
}
|
||||
|
@ -7,8 +7,6 @@
|
||||
#include <Common/COW.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -38,6 +36,11 @@ struct DataTypeWithConstInfo
|
||||
|
||||
using DataTypesWithConstInfo = std::vector<DataTypeWithConstInfo>;
|
||||
|
||||
class SerializationInfo;
|
||||
using SerializationInfoPtr = std::shared_ptr<const SerializationInfo>;
|
||||
using MutableSerializationInfoPtr = std::shared_ptr<SerializationInfo>;
|
||||
struct SerializationInfoSettings;
|
||||
|
||||
/** Properties of data type.
|
||||
*
|
||||
* Contains methods for getting serialization instances.
|
||||
@ -117,7 +120,7 @@ public:
|
||||
|
||||
Names getSubcolumnNames() const;
|
||||
|
||||
virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const;
|
||||
virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfoSettings & settings) const;
|
||||
virtual SerializationInfoPtr getSerializationInfo(const IColumn & column) const;
|
||||
|
||||
/// TODO: support more types.
|
||||
|
@ -13,6 +13,7 @@ namespace DB
|
||||
|
||||
struct StorageSnapshot;
|
||||
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
|
||||
class ColumnsDescription;
|
||||
|
||||
/// Returns number of dimensions in Array type. 0 if type is not array.
|
||||
size_t getNumberOfDimensions(const IDataType & type);
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <DataTypes/Serializations/SerializationInfoSettings.h>
|
||||
|
||||
#include <Poco/JSON/Object.h>
|
||||
|
||||
|
||||
@ -28,6 +30,8 @@ constexpr auto SERIALIZATION_INFO_VERSION = 0;
|
||||
class SerializationInfo
|
||||
{
|
||||
public:
|
||||
using Settings = SerializationInfoSettings;
|
||||
|
||||
struct Data
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
@ -38,16 +42,8 @@ public:
|
||||
void addDefaults(size_t length);
|
||||
};
|
||||
|
||||
struct Settings
|
||||
{
|
||||
const double ratio_of_defaults_for_sparse = 1.0;
|
||||
const bool choose_kind = false;
|
||||
|
||||
bool isAlwaysDefault() const { return ratio_of_defaults_for_sparse >= 1.0; }
|
||||
};
|
||||
|
||||
SerializationInfo(ISerialization::Kind kind_, const Settings & settings_);
|
||||
SerializationInfo(ISerialization::Kind kind_, const Settings & settings_, const Data & data_);
|
||||
SerializationInfo(ISerialization::Kind kind_, const SerializationInfoSettings & settings_);
|
||||
SerializationInfo(ISerialization::Kind kind_, const SerializationInfoSettings & settings_, const Data & data_);
|
||||
|
||||
virtual ~SerializationInfo() = default;
|
||||
|
||||
@ -64,7 +60,7 @@ public:
|
||||
virtual std::shared_ptr<SerializationInfo> createWithType(
|
||||
const IDataType & old_type,
|
||||
const IDataType & new_type,
|
||||
const Settings & new_settings) const;
|
||||
const SerializationInfoSettings & new_settings) const;
|
||||
|
||||
virtual void serialializeKindBinary(WriteBuffer & out) const;
|
||||
virtual void deserializeFromKindsBinary(ReadBuffer & in);
|
||||
@ -73,14 +69,14 @@ public:
|
||||
virtual void fromJSON(const Poco::JSON::Object & object);
|
||||
|
||||
void setKind(ISerialization::Kind kind_) { kind = kind_; }
|
||||
const Settings & getSettings() const { return settings; }
|
||||
const SerializationInfoSettings & getSettings() const { return settings; }
|
||||
const Data & getData() const { return data; }
|
||||
ISerialization::Kind getKind() const { return kind; }
|
||||
|
||||
static ISerialization::Kind chooseKind(const Data & data, const Settings & settings);
|
||||
static ISerialization::Kind chooseKind(const Data & data, const SerializationInfoSettings & settings);
|
||||
|
||||
protected:
|
||||
const Settings settings;
|
||||
const SerializationInfoSettings settings;
|
||||
|
||||
ISerialization::Kind kind;
|
||||
Data data;
|
||||
@ -96,7 +92,7 @@ using MutableSerializationInfos = std::vector<MutableSerializationInfoPtr>;
|
||||
class SerializationInfoByName : public std::map<String, MutableSerializationInfoPtr>
|
||||
{
|
||||
public:
|
||||
using Settings = SerializationInfo::Settings;
|
||||
using Settings = SerializationInfoSettings;
|
||||
|
||||
SerializationInfoByName() = default;
|
||||
SerializationInfoByName(const NamesAndTypesList & columns, const Settings & settings);
|
||||
|
14
src/DataTypes/Serializations/SerializationInfoSettings.h
Normal file
14
src/DataTypes/Serializations/SerializationInfoSettings.h
Normal file
@ -0,0 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct SerializationInfoSettings
|
||||
{
|
||||
const double ratio_of_defaults_for_sparse = 1.0;
|
||||
const bool choose_kind = false;
|
||||
|
||||
bool isAlwaysDefault() const { return ratio_of_defaults_for_sparse >= 1.0; }
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <filesystem>
|
||||
#include <Interpreters/DDLTask.h>
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Databases/DDLDependencyVisitor.h>
|
||||
#include <Databases/DDLLoadingDependencyVisitor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
|
@ -5,8 +5,9 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ApplyWithSubqueryVisitor.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Databases/TablesDependencyGraph.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/DDLTask.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
|
@ -1,11 +1,12 @@
|
||||
#include <memory>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
#include <Interpreters/TableNameHints.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageTransaction.h>
|
||||
#include <Disks/FakeDiskTransaction.h>
|
||||
|
@ -23,6 +23,7 @@ namespace DB
|
||||
class Block;
|
||||
struct Settings;
|
||||
struct FormatFactorySettings;
|
||||
struct ReadSettings;
|
||||
|
||||
class ReadBuffer;
|
||||
class WriteBuffer;
|
||||
|
@ -8,6 +8,7 @@
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class Block;
|
||||
|
||||
/// Extracts information about where the format schema file is from passed context and keep it.
|
||||
class FormatSchemaInfo
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/WithFileSize.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
@ -3,9 +3,15 @@
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Block;
|
||||
class NamesAndTypesList;
|
||||
using NamesAndTypesLists = std::vector<NamesAndTypesList>;
|
||||
|
||||
/// Struct with some additional information about inferred types for JSON formats.
|
||||
struct JSONInferenceInfo
|
||||
{
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
@ -1,12 +1,13 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Storages/StorageJoin.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -274,7 +274,7 @@ public:
|
||||
const typename ColumnIPv6::Container & data = col_from_ip->getData();
|
||||
const auto size = col_from_ip->size();
|
||||
auto & chars_to = col_to->getChars();
|
||||
const auto length = IPV6_BINARY_LENGTH;
|
||||
const auto length = sizeof(IPv6::UnderlyingType);
|
||||
chars_to.resize(size * Impl::length);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <base/range.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <base/range.h>
|
||||
#include <base/TypeLists.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Columns/ColumnsDateTime.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Storages/getStructureOfRemoteTable.h>
|
||||
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
|
||||
#include <list>
|
||||
#include <optional>
|
||||
|
||||
namespace DB
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <base/range.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/PageCache.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Common/PageCache.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -154,9 +154,12 @@ inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf)
|
||||
size_t size = 0;
|
||||
readVarUInt(size, buf);
|
||||
|
||||
if (size != IPV6_BINARY_LENGTH)
|
||||
throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH,
|
||||
"Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH);
|
||||
if (size != sizeof(IPv6::UnderlyingType))
|
||||
throw Exception(
|
||||
ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH,
|
||||
"Size of the string {} doesn't match size of binary IPv6 {}",
|
||||
size,
|
||||
sizeof(IPv6::UnderlyingType));
|
||||
|
||||
buf.readStrict(reinterpret_cast<char*>(&ip.toUnderType()), size);
|
||||
}
|
||||
|
@ -14,6 +14,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct ReadSettings;
|
||||
class SeekableReadBuffer;
|
||||
|
||||
using CreateReadBuffer = std::function<std::unique_ptr<SeekableReadBuffer>()>;
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <base/types.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
|
||||
#include <Poco/Util/MapConfiguration.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
int main()
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include "ActionLocksManager.h"
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
@ -44,6 +44,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
@ -1,13 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <string_view>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <IO/copyData.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Joins.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -19,9 +19,8 @@
|
||||
#include <Disks/IO/getThreadPoolReader.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
@ -149,6 +148,18 @@ template <class Queue>
|
||||
class MergeTreeBackgroundExecutor;
|
||||
class AsyncLoader;
|
||||
|
||||
struct TemporaryTableHolder;
|
||||
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
|
||||
|
||||
class LoadTask;
|
||||
using LoadTaskPtr = std::shared_ptr<LoadTask>;
|
||||
using LoadTaskPtrs = std::vector<LoadTaskPtr>;
|
||||
|
||||
class IClassifier;
|
||||
using ClassifierPtr = std::shared_ptr<IClassifier>;
|
||||
class IResourceManager;
|
||||
using ResourceManagerPtr = std::shared_ptr<IResourceManager>;
|
||||
|
||||
/// Scheduling policy can be changed using `background_merges_mutations_scheduling_policy` config option.
|
||||
/// By default concurrent merges are scheduled using "round_robin" to ensure fair and starvation-free operation.
|
||||
/// Previously in heavily overloaded shards big merges could possibly be starved by smaller
|
||||
|
@ -2,6 +2,8 @@
|
||||
#include <base/sort.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -14,7 +16,6 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTQueryWithTableAndOutput.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <mutex>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/TableNameHints.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
@ -1705,4 +1706,43 @@ DDLGuard::~DDLGuard()
|
||||
releaseTableLock();
|
||||
}
|
||||
|
||||
std::pair<String, String> TableNameHints::getHintForTable(const String & table_name) const
|
||||
{
|
||||
auto results = this->getHints(table_name, getAllRegisteredNames());
|
||||
if (results.empty())
|
||||
return getExtendedHintForTable(table_name);
|
||||
return std::make_pair(database->getDatabaseName(), results[0]);
|
||||
}
|
||||
|
||||
std::pair<String, String> TableNameHints::getExtendedHintForTable(const String & table_name) const
|
||||
{
|
||||
/// load all available databases from the DatabaseCatalog instance
|
||||
auto & database_catalog = DatabaseCatalog::instance();
|
||||
auto all_databases = database_catalog.getDatabases();
|
||||
|
||||
for (const auto & [db_name, db] : all_databases)
|
||||
{
|
||||
/// this case should be covered already by getHintForTable
|
||||
if (db_name == database->getDatabaseName())
|
||||
continue;
|
||||
|
||||
TableNameHints hints(db, context);
|
||||
auto results = hints.getHints(table_name);
|
||||
|
||||
/// if the results are not empty, return the first instance of the table_name
|
||||
/// and the corresponding database_name that was found.
|
||||
if (!results.empty())
|
||||
return std::make_pair(db_name, results[0]);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
Names TableNameHints::getAllRegisteredNames() const
|
||||
{
|
||||
Names result;
|
||||
if (database)
|
||||
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
|
||||
result.emplace_back(table_it->name());
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
@ -1,13 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Databases/TablesDependencyGraph.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
@ -365,68 +363,6 @@ private:
|
||||
static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5;
|
||||
};
|
||||
|
||||
class TableNameHints : public IHints<>
|
||||
{
|
||||
public:
|
||||
TableNameHints(ConstDatabasePtr database_, ContextPtr context_)
|
||||
: context(context_),
|
||||
database(database_)
|
||||
{
|
||||
}
|
||||
|
||||
/// getHintForTable tries to get a hint for the provided table_name in the provided
|
||||
/// database. If the results are empty, it goes for extended hints for the table
|
||||
/// with getExtendedHintForTable which looks for the table name in every database that's
|
||||
/// available in the database catalog. It finally returns a single hint which is the database
|
||||
/// name and table_name pair which is similar to the table_name provided. Perhaps something to
|
||||
/// consider is should we return more than one pair of hint?
|
||||
std::pair<String, String> getHintForTable(const String & table_name) const
|
||||
{
|
||||
auto results = this->getHints(table_name, getAllRegisteredNames());
|
||||
if (results.empty())
|
||||
return getExtendedHintForTable(table_name);
|
||||
return std::make_pair(database->getDatabaseName(), results[0]);
|
||||
}
|
||||
|
||||
/// getExtendedHintsForTable tries to get hint for the given table_name across all
|
||||
/// the databases that are available in the database catalog.
|
||||
std::pair<String, String> getExtendedHintForTable(const String & table_name) const
|
||||
{
|
||||
/// load all available databases from the DatabaseCatalog instance
|
||||
auto & database_catalog = DatabaseCatalog::instance();
|
||||
auto all_databases = database_catalog.getDatabases();
|
||||
|
||||
for (const auto & [db_name, db] : all_databases)
|
||||
{
|
||||
/// this case should be covered already by getHintForTable
|
||||
if (db_name == database->getDatabaseName())
|
||||
continue;
|
||||
|
||||
TableNameHints hints(db, context);
|
||||
auto results = hints.getHints(table_name);
|
||||
|
||||
/// if the results are not empty, return the first instance of the table_name
|
||||
/// and the corresponding database_name that was found.
|
||||
if (!results.empty())
|
||||
return std::make_pair(db_name, results[0]);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
Names getAllRegisteredNames() const override
|
||||
{
|
||||
Names result;
|
||||
if (database)
|
||||
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
|
||||
result.emplace_back(table_it->name());
|
||||
return result;
|
||||
}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
ConstDatabasePtr database;
|
||||
};
|
||||
|
||||
|
||||
/// This class is useful when creating a table or database.
|
||||
/// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog.
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Processors/Chunk.h>
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterCreateIndexQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterCreateIndexQuery.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterDropIndexQuery.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterExistsQuery.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Processors/Transforms/buildPushingToViewsChain.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterWatchQuery.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Parsers/ASTKillQueryQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTOptimizeQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterOptimizeQuery.h>
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTRenameQuery.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterShowFunctionsQuery.h>
|
||||
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterShowFunctionsQuery.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Parsers/ASTShowFunctionsQuery.h>
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterUndropQuery.h>
|
||||
|
@ -12,9 +12,10 @@ limitations under the License. */
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTWatchQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterWatchQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <QueryPipeline/StreamLocalLimits.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <Poco/String.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Interpreters/BlobStorageLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/CrashLog.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/FilesystemCacheLog.h>
|
||||
#include <Interpreters/FilesystemReadPrefetchesLog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
|
@ -1,8 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Common/SystemLogBase.h>
|
||||
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
39
src/Interpreters/TableNameHints.h
Normal file
39
src/Interpreters/TableNameHints.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDatabase;
|
||||
using ConstDatabasePtr = std::shared_ptr<const IDatabase>;
|
||||
|
||||
class TableNameHints : public IHints<>
|
||||
{
|
||||
public:
|
||||
TableNameHints(ConstDatabasePtr database_, ContextPtr context_) : context(context_), database(database_) { }
|
||||
|
||||
/// getHintForTable tries to get a hint for the provided table_name in the provided
|
||||
/// database. If the results are empty, it goes for extended hints for the table
|
||||
/// with getExtendedHintForTable which looks for the table name in every database that's
|
||||
/// available in the database catalog. It finally returns a single hint which is the database
|
||||
/// name and table_name pair which is similar to the table_name provided. Perhaps something to
|
||||
/// consider is should we return more than one pair of hint?
|
||||
std::pair<String, String> getHintForTable(const String & table_name) const;
|
||||
|
||||
/// getExtendedHintsForTable tries to get hint for the given table_name across all
|
||||
/// the databases that are available in the database catalog.
|
||||
std::pair<String, String> getExtendedHintForTable(const String & table_name) const;
|
||||
|
||||
Names getAllRegisteredNames() const override;
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
ConstDatabasePtr database;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Interpreters/DDLTask.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
@ -10,9 +10,10 @@
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,11 +6,12 @@
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterSystemQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/TablesLoader.h>
|
||||
|
@ -51,6 +51,7 @@
|
||||
|
||||
#include <Interpreters/ArrayJoinAction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user