mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
commit
e620ea15bd
@ -357,6 +357,7 @@ void Server::createServer(const std::string & listen_host, const char * port_nam
|
|||||||
try
|
try
|
||||||
{
|
{
|
||||||
func(port);
|
func(port);
|
||||||
|
global_context->registerServerPort(port_name, port);
|
||||||
}
|
}
|
||||||
catch (const Poco::Exception &)
|
catch (const Poco::Exception &)
|
||||||
{
|
{
|
||||||
|
136
src/Functions/getServerPort.cpp
Normal file
136
src/Functions/getServerPort.cpp
Normal file
@ -0,0 +1,136 @@
|
|||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
class ExecutableFunctionGetServerPort : public IExecutableFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ExecutableFunctionGetServerPort(UInt16 port_) : port(port_) {}
|
||||||
|
|
||||||
|
String getName() const override { return "getServerPort"; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
|
||||||
|
{
|
||||||
|
return DataTypeNumber<UInt16>().createColumnConst(input_rows_count, port);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
UInt16 port;
|
||||||
|
};
|
||||||
|
|
||||||
|
class FunctionBaseGetServerPort : public IFunctionBase
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit FunctionBaseGetServerPort(bool is_distributed_, UInt16 port_, DataTypes argument_types_, DataTypePtr return_type_)
|
||||||
|
: is_distributed(is_distributed_), port(port_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return "getServerPort"; }
|
||||||
|
|
||||||
|
const DataTypes & getArgumentTypes() const override
|
||||||
|
{
|
||||||
|
return argument_types;
|
||||||
|
}
|
||||||
|
|
||||||
|
const DataTypePtr & getResultType() const override
|
||||||
|
{
|
||||||
|
return return_type;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isDeterministic() const override { return false; }
|
||||||
|
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||||
|
bool isSuitableForConstantFolding() const override { return !is_distributed; }
|
||||||
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||||
|
|
||||||
|
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||||
|
{
|
||||||
|
return std::make_unique<ExecutableFunctionGetServerPort>(port);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
bool is_distributed;
|
||||||
|
UInt16 port;
|
||||||
|
DataTypes argument_types;
|
||||||
|
DataTypePtr return_type;
|
||||||
|
};
|
||||||
|
|
||||||
|
class GetServerPortOverloadResolver : public IFunctionOverloadResolver, WithContext
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "getServerPort";
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
static FunctionOverloadResolverPtr create(ContextPtr context_)
|
||||||
|
{
|
||||||
|
return std::make_unique<GetServerPortOverloadResolver>(context_);
|
||||||
|
}
|
||||||
|
|
||||||
|
explicit GetServerPortOverloadResolver(ContextPtr context_) : WithContext(context_) {}
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
|
||||||
|
bool isDeterministic() const override { return false; }
|
||||||
|
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override
|
||||||
|
{
|
||||||
|
size_t number_of_arguments = data_types.size();
|
||||||
|
if (number_of_arguments != 1)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||||
|
"Number of arguments for function {} doesn't match: passed {}, should be 1",
|
||||||
|
getName(),
|
||||||
|
number_of_arguments);
|
||||||
|
return std::make_shared<DataTypeNumber<UInt16>>();
|
||||||
|
}
|
||||||
|
|
||||||
|
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||||
|
{
|
||||||
|
if (!isString(arguments[0].type))
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
|
"The argument of function {} should be a constant string with the name of a setting",
|
||||||
|
getName());
|
||||||
|
const auto * column = arguments[0].column.get();
|
||||||
|
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN,
|
||||||
|
"The argument of function {} should be a constant string with the name of a setting",
|
||||||
|
getName());
|
||||||
|
|
||||||
|
String port_name{column->getDataAt(0)};
|
||||||
|
auto port = getContext()->getServerPort(port_name);
|
||||||
|
|
||||||
|
DataTypes argument_types;
|
||||||
|
argument_types.emplace_back(arguments.back().type);
|
||||||
|
return std::make_unique<FunctionBaseGetServerPort>(getContext()->isDistributed(), port, argument_types, return_type);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerFunctionGetServerPort(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<GetServerPortOverloadResolver>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -71,6 +71,7 @@ void registerFunctionHasThreadFuzzer(FunctionFactory &);
|
|||||||
void registerFunctionInitializeAggregation(FunctionFactory &);
|
void registerFunctionInitializeAggregation(FunctionFactory &);
|
||||||
void registerFunctionErrorCodeToName(FunctionFactory &);
|
void registerFunctionErrorCodeToName(FunctionFactory &);
|
||||||
void registerFunctionTcpPort(FunctionFactory &);
|
void registerFunctionTcpPort(FunctionFactory &);
|
||||||
|
void registerFunctionGetServerPort(FunctionFactory &);
|
||||||
void registerFunctionByteSize(FunctionFactory &);
|
void registerFunctionByteSize(FunctionFactory &);
|
||||||
void registerFunctionFile(FunctionFactory & factory);
|
void registerFunctionFile(FunctionFactory & factory);
|
||||||
void registerFunctionConnectionId(FunctionFactory & factory);
|
void registerFunctionConnectionId(FunctionFactory & factory);
|
||||||
@ -150,6 +151,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|||||||
registerFunctionInitializeAggregation(factory);
|
registerFunctionInitializeAggregation(factory);
|
||||||
registerFunctionErrorCodeToName(factory);
|
registerFunctionErrorCodeToName(factory);
|
||||||
registerFunctionTcpPort(factory);
|
registerFunctionTcpPort(factory);
|
||||||
|
registerFunctionGetServerPort(factory);
|
||||||
registerFunctionByteSize(factory);
|
registerFunctionByteSize(factory);
|
||||||
registerFunctionFile(factory);
|
registerFunctionFile(factory);
|
||||||
registerFunctionConnectionId(factory);
|
registerFunctionConnectionId(factory);
|
||||||
|
@ -228,6 +228,8 @@ struct ContextSharedPart
|
|||||||
ConfigurationPtr clusters_config; /// Stores updated configs
|
ConfigurationPtr clusters_config; /// Stores updated configs
|
||||||
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
|
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
|
||||||
|
|
||||||
|
std::map<String, UInt16> server_ports;
|
||||||
|
|
||||||
bool shutdown_called = false;
|
bool shutdown_called = false;
|
||||||
|
|
||||||
Stopwatch uptime_watch;
|
Stopwatch uptime_watch;
|
||||||
@ -1798,6 +1800,20 @@ std::optional<UInt16> Context::getTCPPortSecure() const
|
|||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Context::registerServerPort(String port_name, UInt16 port)
|
||||||
|
{
|
||||||
|
shared->server_ports.emplace(std::move(port_name), port);
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt16 Context::getServerPort(const String & port_name) const
|
||||||
|
{
|
||||||
|
auto it = shared->server_ports.find(port_name);
|
||||||
|
if (it == shared->server_ports.end())
|
||||||
|
throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name);
|
||||||
|
else
|
||||||
|
return it->second;
|
||||||
|
}
|
||||||
|
|
||||||
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
|
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
|
||||||
{
|
{
|
||||||
auto res = getClusters()->getCluster(cluster_name);
|
auto res = getClusters()->getCluster(cluster_name);
|
||||||
|
@ -580,6 +580,11 @@ public:
|
|||||||
|
|
||||||
std::optional<UInt16> getTCPPortSecure() const;
|
std::optional<UInt16> getTCPPortSecure() const;
|
||||||
|
|
||||||
|
/// Register server ports during server starting up. No lock is held.
|
||||||
|
void registerServerPort(String port_name, UInt16 port);
|
||||||
|
|
||||||
|
UInt16 getServerPort(const String & port_name) const;
|
||||||
|
|
||||||
/// For methods below you may need to acquire the context lock by yourself.
|
/// For methods below you may need to acquire the context lock by yourself.
|
||||||
|
|
||||||
ContextMutablePtr getQueryContext() const;
|
ContextMutablePtr getQueryContext() const;
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
9000
|
3
tests/queries/0_stateless/02012_get_server_port.sql
Normal file
3
tests/queries/0_stateless/02012_get_server_port.sql
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
select getServerPort('tcp_port');
|
||||||
|
|
||||||
|
select getServerPort('unknown'); -- { serverError 170 }
|
Loading…
Reference in New Issue
Block a user