Merge branch 'master' into generic-clickhouse-local-fuzzer

This commit is contained in:
Alexey Milovidov 2024-03-21 03:18:05 +01:00
commit 784de2dca3
147 changed files with 8669 additions and 1629 deletions

View File

@ -61,11 +61,16 @@ if (ENABLE_CHECK_HEAVY_BUILDS)
# set CPU time limit to 1000 seconds
set (RLIMIT_CPU 1000)
# -fsanitize=memory and address are too heavy
# Sanitizers are too heavy
if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE)
set (RLIMIT_DATA 10000000000) # 10G
endif()
# For some files currently building RISCV64 might be too slow. TODO: Improve compilation times per file
if (ARCH_RISCV64)
set (RLIMIT_CPU 1800)
endif()
set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER})
endif ()

View File

@ -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)));
}
};

View File

@ -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);

View File

@ -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;

View File

@ -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>

View File

@ -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>

View File

@ -13,6 +13,8 @@
#include <optional>
#include <vector>
#include <boost/noncopyable.hpp>
namespace Poco { class Logger; }
namespace Poco::Net { class IPAddress; }

View File

@ -1,4 +1,5 @@
#include <Access/User.h>
#include <Common/StringUtils/StringUtils.h>
#include <Core/Protocol.h>
#include <base/insertAtEnd.h>

View File

@ -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

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -1,6 +1,7 @@
#pragma once
#include <Backups/BackupIO.h>
#include <Common/Logger.h>
#include <IO/ReadSettings.h>
#include <IO/WriteSettings.h>

View File

@ -1,7 +1,9 @@
#pragma once
#include <Backups/BackupIO_Default.h>
#include <Common/Logger.h>
#include <Disks/DiskType.h>
#include <filesystem>

View File

@ -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>

View File

@ -8,6 +8,7 @@
#include <base/argsToConfig.h>
#include <base/safeExit.h>
#include <Core/Block.h>
#include <Core/BaseSettingsProgramOptions.h>
#include <Core/Protocol.h>
#include <Common/DateLUT.h>
#include <Common/MemoryTracker.h>
@ -2753,9 +2754,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)
{
@ -2776,9 +2777,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();

View File

@ -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

View File

@ -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"

View File

@ -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

View 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);
}
}

View File

@ -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>

View File

@ -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;
};
}

View File

@ -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:

View File

@ -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>

View File

@ -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;

View File

@ -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

View File

@ -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)
{

View 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
View File

@ -0,0 +1,16 @@
#pragma once
namespace DB
{
enum class LogsLevel
{
none = 0, /// Disable
fatal,
error,
warning,
information,
debug,
trace,
test,
};
}

View File

@ -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)

View File

@ -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>

View File

@ -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
{

View File

@ -1,6 +1,5 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <Parsers/IAST_fwd.h>
#include <Common/IFactoryWithAliases.h>
#include <DataTypes/DataTypeCustom.h>

View File

@ -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());

View File

@ -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]; }

View File

@ -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);
}

View File

@ -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.

View File

@ -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);

View File

@ -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);

View 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; }
};
}

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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

View File

@ -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>

View File

@ -23,6 +23,7 @@ namespace DB
class Block;
struct Settings;
struct FormatFactorySettings;
struct ReadSettings;
class ReadBuffer;
class WriteBuffer;

View File

@ -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

View File

@ -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>

View File

@ -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
{

View File

@ -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>

View File

@ -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

View File

@ -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)
{

View File

@ -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

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -7,6 +7,7 @@
#include <Storages/IStorage.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Storages/getStructureOfRemoteTable.h>

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnsNumber.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <list>
#include <optional>
namespace DB

View File

@ -4,6 +4,8 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <base/range.h>
namespace DB
{
namespace ErrorCodes

View File

@ -1,7 +1,8 @@
#pragma once
#include <Common/PageCache.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadSettings.h>
#include <Common/PageCache.h>
namespace DB
{

View File

@ -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);
}

View File

@ -14,6 +14,7 @@
namespace DB
{
struct ReadSettings;
class SeekableReadBuffer;
using CreateReadBuffer = std::function<std::unique_ptr<SeekableReadBuffer>()>;

View File

@ -6,6 +6,8 @@
#include <base/types.h>
#include <Common/Config/ConfigProcessor.h>
#include <Poco/Util/MapConfiguration.h>
using namespace DB;
int main()

View File

@ -1,5 +1,6 @@
#include "ActionLocksManager.h"
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Databases/IDatabase.h>
#include <Storages/IStorage.h>

View File

@ -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>

View File

@ -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
{

View File

@ -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>

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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;
}
}

View File

@ -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.

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <Core/Names.h>
#include <Processors/Chunk.h>

View File

@ -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>

View File

@ -16,6 +16,7 @@
#include <DataTypes/DataTypeString.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/ProcessList.h>
#include <Parsers/ASTCheckQuery.h>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -1,4 +1,5 @@
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/InterpreterUndropQuery.h>

View File

@ -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>

View File

@ -2,6 +2,7 @@
#include <Core/SettingsEnums.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>

View File

@ -4,6 +4,8 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Poco/String.h>
namespace DB
{

View File

@ -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>

View File

@ -1,8 +1,9 @@
#pragma once
#include <Interpreters/StorageID.h>
#include <Common/SystemLogBase.h>
#include <Interpreters/StorageID.h>
#include <boost/noncopyable.hpp>
namespace DB
{

View 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;
};
}

View File

@ -1,4 +1,5 @@
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>

View File

@ -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>

View File

@ -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
{

View File

@ -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>

Some files were not shown because too many files have changed in this diff Show More