mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Less indirect dependencies on Settings.h
This commit is contained in:
parent
56f497bf7f
commit
a41bbc5b3d
@ -8,6 +8,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/FunctionDocumentation.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
|
||||
@ -354,6 +355,11 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
FunctionPtr FunctionGenerateRandomStructure::create(DB::ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionGenerateRandomStructure>(context->getSettingsRef().allow_suspicious_low_cardinality_types.value);
|
||||
}
|
||||
|
||||
DataTypePtr FunctionGenerateRandomStructure::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() > 2)
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
@ -18,10 +17,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionGenerateRandomStructure>(context->getSettingsRef().allow_suspicious_low_cardinality_types.value);
|
||||
}
|
||||
static FunctionPtr create(ContextPtr context);
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
|
13
src/Functions/FunctionUnixTimestamp64.cpp
Normal file
13
src/Functions/FunctionUnixTimestamp64.cpp
Normal file
@ -0,0 +1,13 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Functions/FunctionUnixTimestamp64.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
FunctionFromUnixTimestamp64::FunctionFromUnixTimestamp64(size_t target_scale_, const char * name_, ContextPtr context)
|
||||
: target_scale(target_scale_)
|
||||
, name(name_)
|
||||
, allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments)
|
||||
{}
|
||||
|
||||
}
|
@ -6,7 +6,6 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <base/arithmeticOverflow.h>
|
||||
@ -108,11 +107,7 @@ private:
|
||||
const bool allow_nonconst_timezone_arguments;
|
||||
|
||||
public:
|
||||
FunctionFromUnixTimestamp64(size_t target_scale_, const char * name_, ContextPtr context)
|
||||
: target_scale(target_scale_)
|
||||
, name(name_)
|
||||
, allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments)
|
||||
{}
|
||||
FunctionFromUnixTimestamp64(size_t target_scale_, const char * name_, ContextPtr context);
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
@ -1,8 +1,15 @@
|
||||
#include <Functions/tuple.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
FunctionPtr FunctionTuple::create(DB::ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionTuple>(context->getSettingsRef().enable_named_columns_in_function_tuple);
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(Tuple)
|
||||
{
|
||||
factory.registerFunction<FunctionTuple>(FunctionDocumentation{
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -22,11 +21,7 @@ class FunctionTuple : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "tuple";
|
||||
|
||||
/// maybe_unused: false-positive
|
||||
[[maybe_unused]] static FunctionPtr create(ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionTuple>(context->getSettingsRef().enable_named_columns_in_function_tuple);
|
||||
}
|
||||
static FunctionPtr create(ContextPtr context);
|
||||
|
||||
explicit FunctionTuple(bool enable_named_columns_ = false) : enable_named_columns(enable_named_columns_) { }
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Functions/grouping.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/FailPoint.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Planner/Utils.h>
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperLock.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <base/sleep.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Interpreters/IInterpreterUnionOrSelectQuery.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
@ -15,6 +17,23 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const DB::ASTPtr& query_ptr_,
|
||||
const DB::ContextMutablePtr& context_, const DB::SelectQueryOptions& options_)
|
||||
: query_ptr(query_ptr_)
|
||||
, context(context_)
|
||||
, options(options_)
|
||||
, max_streams(context->getSettingsRef().max_threads)
|
||||
{
|
||||
if (options.shard_num)
|
||||
context->addSpecialScalar(
|
||||
"_shard_num",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}});
|
||||
if (options.shard_count)
|
||||
context->addSpecialScalar(
|
||||
"_shard_count",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared<DataTypeUInt32>(), "_shard_count"}});
|
||||
}
|
||||
|
||||
QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline()
|
||||
{
|
||||
QueryPlan query_plan;
|
||||
|
@ -6,8 +6,6 @@
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -19,21 +17,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextMutablePtr & context_, const SelectQueryOptions & options_)
|
||||
: query_ptr(query_ptr_)
|
||||
, context(context_)
|
||||
, options(options_)
|
||||
, max_streams(context->getSettingsRef().max_threads)
|
||||
{
|
||||
if (options.shard_num)
|
||||
context->addSpecialScalar(
|
||||
"_shard_num",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}});
|
||||
if (options.shard_count)
|
||||
context->addSpecialScalar(
|
||||
"_shard_count",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared<DataTypeUInt32>(), "_shard_count"}});
|
||||
}
|
||||
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextMutablePtr & context_, const SelectQueryOptions & options_);
|
||||
|
||||
virtual void buildQueryPlan(QueryPlan & query_plan) = 0;
|
||||
QueryPipelineBuilder buildQueryPipeline();
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
@ -24,7 +26,6 @@
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <Storages/StorageKeeperMap.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Access/User.h>
|
||||
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -14,6 +14,7 @@
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/PoolId.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <base/hex.h>
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
|
@ -29,6 +29,7 @@
|
||||
#include <QueryPipeline/printPipeline.h>
|
||||
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/QueryTreePassManager.h>
|
||||
|
@ -60,6 +60,7 @@
|
||||
#include <Parsers/ASTExternalDDLQuery.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Access/EnabledQuota.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Processors/Transforms/buildPushingToViewsChain.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Access/AccessControl.h>
|
||||
|
||||
#include <Columns/getLeastSuperColumn.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectIntersectExceptQuery.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
|
@ -84,6 +84,7 @@
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Access/AccessControl.h>
|
||||
|
||||
#include <Columns/getLeastSuperColumn.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectIntersectExceptQuery.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/JoinedTables.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/DistributedCreateLocalPlan.h>
|
||||
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
|
@ -22,9 +22,10 @@
|
||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
#include <Processors/QueryPlan/WindowStep.h>
|
||||
#include "Storages/KeyDescription.h"
|
||||
#include <Storages/KeyDescription.h>
|
||||
#include <Storages/StorageMerge.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <stack>
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Analyzer/QueryNode.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/StorageDummy.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Planner/PlannerExpressionAnalysis.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
#include <Functions/FunctionsLogical.h>
|
||||
|
@ -41,6 +41,7 @@
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Processors/QueryPlan/ReadFromStreamLikeEngine.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <fmt/format.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/IStorageCluster.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
|
@ -27,6 +27,7 @@ limitations under the License. */
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <base/hex.h>
|
||||
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
@ -7,6 +7,7 @@
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/ActionBlocker.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Processors/Transforms/CheckSortedTransform.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
|
@ -37,6 +37,7 @@
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/Statistics/Statistics.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/MergeTree/MergeAlgorithm.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
@ -575,6 +576,16 @@ bool ReplicatedMergeTreeSinkImpl<false>::writeExistingPart(MergeTreeData::Mutabl
|
||||
}
|
||||
}
|
||||
|
||||
template<bool async_insert>
|
||||
bool ReplicatedMergeTreeSinkImpl<async_insert>::lastBlockIsDuplicate() const
|
||||
{
|
||||
/// If MV is responsible for deduplication, block is not considered duplicating.
|
||||
if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
||||
return false;
|
||||
|
||||
return last_block_is_duplicate;
|
||||
}
|
||||
|
||||
template<bool async_insert>
|
||||
std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::detectConflictsInAsyncBlockIDs(const std::vector<String> & ids)
|
||||
{
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <base/types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperRetries.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperWithFaultInjection.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
|
||||
|
||||
@ -61,14 +60,7 @@ public:
|
||||
bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part);
|
||||
|
||||
/// For proper deduplication in MaterializedViews
|
||||
bool lastBlockIsDuplicate() const override
|
||||
{
|
||||
/// If MV is responsible for deduplication, block is not considered duplicating.
|
||||
if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
||||
return false;
|
||||
|
||||
return last_block_is_duplicate;
|
||||
}
|
||||
bool lastBlockIsDuplicate() const override;
|
||||
|
||||
struct DelayedChunk;
|
||||
private:
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/RocksDB/RocksDBSettings.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
|
@ -38,6 +38,7 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "StorageExternalDistributed.h"
|
||||
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
|
@ -53,6 +53,8 @@
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/re2.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/JoinUtils.h>
|
||||
#include <Formats/NativeWriter.h>
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <cassert>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
@ -37,7 +37,6 @@
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
|
||||
|
@ -22,6 +22,7 @@
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/StorageURLCluster.h>
|
||||
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
|
@ -19,6 +19,8 @@
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
@ -48,6 +48,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
|
||||
#include <Storages/LiveView/StorageBlocks.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
Loading…
Reference in New Issue
Block a user