mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #66404 from Algunenano/mergetree_private
Untangle setting headers
This commit is contained in:
commit
e4f348acc7
@ -24,9 +24,8 @@
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -49,6 +48,8 @@
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
using namespace std::literals;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include "LibraryBridge.h"
|
||||
|
||||
#include <iostream>
|
||||
|
||||
int mainEntryClickHouseLibraryBridge(int argc, char ** argv)
|
||||
{
|
||||
DB::LibraryBridge app;
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include "ExternalDictionaryLibraryHandlerFactory.h"
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/BridgeProtocolVersion.h>
|
||||
|
@ -3,11 +3,12 @@
|
||||
#include <Client/ClientBase.h>
|
||||
#include <Client/LocalConnection.h>
|
||||
|
||||
#include <Common/StatusFile.h>
|
||||
#include <Common/InterruptListener.h>
|
||||
#include <Loggers/Loggers.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Loggers/Loggers.h>
|
||||
#include <Common/InterruptListener.h>
|
||||
#include <Common/StatusFile.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <memory>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#if USE_ODBC
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Poco/Net/HTTPServerResponse.h>
|
||||
#include <Common/BridgeProtocolVersion.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include "getIdentifierQuote.h"
|
||||
#include "validateODBCConnectionString.h"
|
||||
#include "ODBCPooledConnectionFactory.h"
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/ReadBufferFromIStream.h>
|
||||
#include <Poco/Net/HTTPServerRequest.h>
|
||||
#include <Poco/Net/HTTPServerResponse.h>
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include "ODBCBridge.h"
|
||||
|
||||
#include <iostream>
|
||||
|
||||
int mainEntryClickHouseODBCBridge(int argc, char ** argv)
|
||||
{
|
||||
DB::ODBCBridge app;
|
||||
|
@ -2,8 +2,10 @@
|
||||
|
||||
#if USE_ODBC
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Server/HTTP/HTMLForm.h>
|
||||
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Net/HTTPServerRequest.h>
|
||||
|
@ -68,6 +68,7 @@
|
||||
#include <Interpreters/registerInterpreters.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
|
@ -21,7 +21,6 @@
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <base/defines.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/re2.h>
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
static constexpr size_t MAX_AGGREGATE_FUNCTION_NAME_LENGTH = 1000;
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Analyzer/UnionNode.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -11,6 +11,8 @@
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -11,6 +11,8 @@
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/LambdaNode.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <Analyzer/TableExpressionModifiers.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -22,6 +22,8 @@
|
||||
#include <Analyzer/HashUtils.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -10,6 +10,8 @@
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/HashUtils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
@ -11,6 +11,8 @@
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -9,13 +9,14 @@
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/logical.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -21,6 +21,8 @@
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Functions/grouping.h>
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/multiIf.h>
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Analyzer/HashUtils.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/if.h>
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/SortNode.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,19 +1,16 @@
|
||||
#include <Analyzer/Passes/SumIfToCountIfPass.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -9,6 +9,8 @@
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -9,6 +9,8 @@
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -40,6 +40,8 @@
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Analyzer/UnionNode.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Databases/IDatabase.h>
|
||||
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
@ -1,8 +1,9 @@
|
||||
#include <Analyzer/Resolve/IdentifierResolveScope.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/UnionNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -25,6 +25,8 @@
|
||||
#include <Analyzer/Resolve/IdentifierResolveScope.h>
|
||||
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
|
@ -64,6 +64,8 @@
|
||||
#include <Analyzer/Resolve/TableExpressionsAliasVisitor.h>
|
||||
#include <Analyzer/Resolve/ReplaceColumnsVisitor.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ScalarSubqueriesGlobalCacheHit;
|
||||
|
@ -10,6 +10,8 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/sleep.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Backups/BackupIO_S3.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include <Backups/DDLAdjustingForBackupVisitor.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -23,8 +23,9 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
|
||||
#include <filesystem>
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include <mutex>
|
||||
#include <Backups/WithRetries.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Common/SensitiveDataMasker.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include "LibraryBridgeHelper.h"
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Core/ExternalTable.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Client/Suggest.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Client/ConnectionEstablisher.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Client/ConnectionPool.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <boost/functional/hash.hpp>
|
||||
|
||||
@ -85,4 +86,15 @@ ConnectionPoolFactory & ConnectionPoolFactory::instance()
|
||||
return ret;
|
||||
}
|
||||
|
||||
IConnectionPool::Entry ConnectionPool::get(const DB::ConnectionTimeouts& timeouts, const DB::Settings& settings,
|
||||
bool force_connected)
|
||||
{
|
||||
Entry entry = Base::get(settings.connection_pool_max_wait_ms.totalMilliseconds());
|
||||
|
||||
if (force_connected)
|
||||
entry->forceConnected(timeouts);
|
||||
|
||||
return entry;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,12 +4,13 @@
|
||||
#include <Common/Priority.h>
|
||||
#include <Client/Connection.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
|
||||
/** Interface for connection pools.
|
||||
*
|
||||
* Usage (using the usual `ConnectionPool` example)
|
||||
@ -102,15 +103,7 @@ public:
|
||||
|
||||
Entry get(const ConnectionTimeouts & timeouts, /// NOLINT
|
||||
const Settings & settings,
|
||||
bool force_connected = true) override
|
||||
{
|
||||
Entry entry = Base::get(settings.connection_pool_max_wait_ms.totalMilliseconds());
|
||||
|
||||
if (force_connected)
|
||||
entry->forceConnected(timeouts);
|
||||
|
||||
return entry;
|
||||
}
|
||||
bool force_connected = true) override;
|
||||
|
||||
std::string getDescription() const
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
#include <Client/HedgedConnections.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
@ -8,13 +8,14 @@
|
||||
#include <Common/Fiber.h>
|
||||
#include <Client/ConnectionEstablisher.h>
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <unordered_map>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
|
||||
/** Class for establishing hedged connections with replicas.
|
||||
* The process of establishing connection is divided on stages, on each stage if
|
||||
* replica doesn't respond for a long time, we start establishing connection with
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <IO/Operators.h>
|
||||
|
@ -2,7 +2,9 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Poco/DigestStream.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/GetPriorityForLoadBalancing.h>
|
||||
#include <Common/Priority.h>
|
||||
|
||||
|
@ -1,6 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Common/Priority.h>
|
||||
#include <Core/LoadBalancing.h>
|
||||
|
||||
#include <functional>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Common/NamedCollections/NamedCollectionsFactory.h>
|
||||
#include <Common/NamedCollections/NamedCollectionConfiguration.h>
|
||||
#include <Common/NamedCollections/NamedCollectionsMetadataStorage.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <base/sleep.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include "ProgressIndication.h"
|
||||
#include <algorithm>
|
||||
#include <cstddef>
|
||||
#include <iostream>
|
||||
#include <numeric>
|
||||
#include <filesystem>
|
||||
#include <cmath>
|
||||
|
@ -1,14 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
#include <mutex>
|
||||
#include <IO/Progress.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/EventRateMeter.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -12,13 +12,14 @@
|
||||
#include <base/sleep.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Common/CurrentMemoryTracker.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include "config.h" // USE_JEMALLOC
|
||||
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Coordination/FourLetterCommand.h>
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Server/KeeperTCPHandler.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
#include <Coordination/KeeperContext.h>
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/Defines.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
#include <Server/CloudPlacementInfo.h>
|
||||
|
@ -5,8 +5,8 @@
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <base/hex.h>
|
||||
#include "Common/ZooKeeper/IKeeper.h"
|
||||
#include "Common/ZooKeeper/ZooKeeperCommon.h"
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
@ -14,6 +14,7 @@
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <functional>
|
||||
#include <Coordination/KeeperServer.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/Keeper4LWInfo.h>
|
||||
#include <Coordination/KeeperConnectionStats.h>
|
||||
#include <Coordination/KeeperSnapshotManagerS3.h>
|
||||
|
@ -3,9 +3,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperLogStore.h>
|
||||
#include <Coordination/KeeperSnapshotManagerS3.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
@ -30,6 +28,10 @@
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
|
||||
#pragma clang diagnostic ignored "-Wdeprecated-declarations"
|
||||
#include <fmt/chrono.h>
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/InMemoryLogStore.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
#include <Coordination/KeeperStateManager.h>
|
||||
@ -17,6 +16,9 @@ namespace DB
|
||||
|
||||
using RaftAppendResult = nuraft::ptr<nuraft::cmd_result<nuraft::ptr<nuraft::buffer>>>;
|
||||
|
||||
struct KeeperConfigurationAndSettings;
|
||||
using KeeperConfigurationAndSettingsPtr = std::shared_ptr<KeeperConfigurationAndSettings>;
|
||||
|
||||
class KeeperServer
|
||||
{
|
||||
private:
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <atomic>
|
||||
#include <cerrno>
|
||||
#include <chrono>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Coordination/KeeperReconfiguration.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Coordination/KeeperSnapshotManagerS3.h>
|
||||
#include <Coordination/KeeperContext.h>
|
||||
|
@ -1,14 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <string>
|
||||
#include <Coordination/KeeperLogStore.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Core/Types.h>
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include "Coordination/KeeperStateMachine.h"
|
||||
#include "Coordination/RaftServerConfig.h"
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -20,11 +20,12 @@
|
||||
#include <Common/LockMemoryExceptionInThread.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperCommon.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Coordination/KeeperReconfiguration.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
|
||||
#include <functional>
|
||||
#include <base/defines.h>
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <filesystem>
|
||||
#include <thread>
|
||||
#include <Coordination/Changelog.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/InMemoryLogStore.h>
|
||||
#include <Coordination/KeeperContext.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
#include <Core/ExternalTable.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Poco/Net/MessageHeader.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
|
27
src/Core/LoadBalancing.h
Normal file
27
src/Core/LoadBalancing.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class LoadBalancing : uint8_t
|
||||
{
|
||||
/// among replicas with a minimum number of errors selected randomly
|
||||
RANDOM = 0,
|
||||
/// a replica is selected among the replicas with the minimum number of errors
|
||||
/// with the minimum number of distinguished characters in the replica name prefix and local hostname prefix
|
||||
NEAREST_HOSTNAME,
|
||||
/// just like NEAREST_HOSTNAME, but it count distinguished characters in a levenshtein distance manner
|
||||
HOSTNAME_LEVENSHTEIN_DISTANCE,
|
||||
// replicas with the same number of errors are accessed in the same order
|
||||
// as they are specified in the configuration.
|
||||
IN_ORDER,
|
||||
/// if first replica one has higher number of errors,
|
||||
/// pick a random one from replicas with minimum number of errors
|
||||
FIRST_OR_RANDOM,
|
||||
// round robin across replicas with the same number of errors.
|
||||
ROUND_ROBIN,
|
||||
};
|
||||
|
||||
}
|
17
src/Core/QueryLogElementType.h
Normal file
17
src/Core/QueryLogElementType.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// Make it signed for compatibility with DataTypeEnum8
|
||||
enum QueryLogElementType : int8_t
|
||||
{
|
||||
QUERY_START = 1,
|
||||
QUERY_FINISH = 2,
|
||||
EXCEPTION_BEFORE_START = 3,
|
||||
EXCEPTION_WHILE_PROCESSING = 4,
|
||||
};
|
||||
|
||||
}
|
14
src/Core/SchemaInferenceMode.h
Normal file
14
src/Core/SchemaInferenceMode.h
Normal file
@ -0,0 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class SchemaInferenceMode : uint8_t
|
||||
{
|
||||
DEFAULT,
|
||||
UNION,
|
||||
};
|
||||
|
||||
}
|
@ -1,8 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Joins.h>
|
||||
#include <Core/LoadBalancing.h>
|
||||
#include <Core/LogsLevel.h>
|
||||
#include <Core/QueryLogElementType.h>
|
||||
#include <Core/SchemaInferenceMode.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Core/ShortCircuitFunctionEvaluation.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Parsers/ASTSQLSecurity.h>
|
||||
@ -115,25 +119,6 @@ constexpr auto getEnumValues();
|
||||
return getEnumValues<EnumType>().size();\
|
||||
}
|
||||
|
||||
enum class LoadBalancing : uint8_t
|
||||
{
|
||||
/// among replicas with a minimum number of errors selected randomly
|
||||
RANDOM = 0,
|
||||
/// a replica is selected among the replicas with the minimum number of errors
|
||||
/// with the minimum number of distinguished characters in the replica name prefix and local hostname prefix
|
||||
NEAREST_HOSTNAME,
|
||||
/// just like NEAREST_HOSTNAME, but it count distinguished characters in a levenshtein distance manner
|
||||
HOSTNAME_LEVENSHTEIN_DISTANCE,
|
||||
// replicas with the same number of errors are accessed in the same order
|
||||
// as they are specified in the configuration.
|
||||
IN_ORDER,
|
||||
/// if first replica one has higher number of errors,
|
||||
/// pick a random one from replicas with minimum number of errors
|
||||
FIRST_OR_RANDOM,
|
||||
// round robin across replicas with the same number of errors.
|
||||
ROUND_ROBIN,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(LoadBalancing)
|
||||
|
||||
DECLARE_SETTING_ENUM(JoinStrictness)
|
||||
@ -204,16 +189,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion)
|
||||
|
||||
DECLARE_SETTING_ENUM(LogsLevel)
|
||||
|
||||
|
||||
// Make it signed for compatibility with DataTypeEnum8
|
||||
enum QueryLogElementType : int8_t
|
||||
{
|
||||
QUERY_START = 1,
|
||||
QUERY_FINISH = 2,
|
||||
EXCEPTION_BEFORE_START = 3,
|
||||
EXCEPTION_WHILE_PROCESSING = 4,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM_WITH_RENAME(LogQueriesType, QueryLogElementType)
|
||||
|
||||
|
||||
@ -292,13 +267,6 @@ enum class StreamingHandleErrorMode : uint8_t
|
||||
|
||||
DECLARE_SETTING_ENUM(StreamingHandleErrorMode)
|
||||
|
||||
enum class ShortCircuitFunctionEvaluation : uint8_t
|
||||
{
|
||||
ENABLE, // Use short-circuit function evaluation for functions that are suitable for it.
|
||||
FORCE_ENABLE, // Use short-circuit function evaluation for all functions.
|
||||
DISABLE, // Disable short-circuit function evaluation.
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation)
|
||||
|
||||
enum class TransactionsWaitCSNMode : uint8_t
|
||||
@ -367,12 +335,6 @@ DECLARE_SETTING_ENUM(ObjectStorageQueueAction)
|
||||
|
||||
DECLARE_SETTING_ENUM(ExternalCommandStderrReaction)
|
||||
|
||||
enum class SchemaInferenceMode : uint8_t
|
||||
{
|
||||
DEFAULT,
|
||||
UNION,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(SchemaInferenceMode)
|
||||
|
||||
DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateTimeOverflowBehavior)
|
||||
|
15
src/Core/ShortCircuitFunctionEvaluation.h
Normal file
15
src/Core/ShortCircuitFunctionEvaluation.h
Normal file
@ -0,0 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class ShortCircuitFunctionEvaluation : uint8_t
|
||||
{
|
||||
ENABLE, // Use short-circuit function evaluation for functions that are suitable for it.
|
||||
FORCE_ENABLE, // Use short-circuit function evaluation for all functions.
|
||||
DISABLE, // Disable short-circuit function evaluation.
|
||||
};
|
||||
|
||||
}
|
@ -1,15 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <cstddef>
|
||||
#include <string>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Common/IntervalKind.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Columns/Collator.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <string>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <base/errnoToString.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include <Daemon/SentryWriter.h>
|
||||
#include <Common/GWPAsan.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <type_traits>
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Common/StringUtils.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodesEnumValues
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Parsers/ParserSelectWithUnionQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Common/KnownObjectNames.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Poco/String.h>
|
||||
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Common/PoolId.h>
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseFilesystem.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/re2.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Poco/URI.h>
|
||||
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <filesystem>
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
@ -30,6 +31,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
|
@ -13,6 +13,8 @@
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/PoolId.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Databases/DatabaseReplicatedWorker.h>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/DDLTask.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <filesystem>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseS3.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <IO/S3/URI.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