mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
commit
0fc1268432
@ -117,6 +117,10 @@ Returns the part of the domain that includes top-level subdomains up to the “f
|
||||
|
||||
For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`.
|
||||
|
||||
### port(URL[, default_port = 0]) {#port}
|
||||
|
||||
Returns the port or `default_port` if there is no port in the URL (or in case of validation error).
|
||||
|
||||
### path {#path}
|
||||
|
||||
Returns the path. Example: `/top/news.html` The path does not include the query string.
|
||||
|
129
src/Functions/URL/port.cpp
Normal file
129
src/Functions/URL/port.cpp
Normal file
@ -0,0 +1,129 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include "domain.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
struct FunctionPort : public IFunction
|
||||
{
|
||||
static constexpr auto name = "port";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionPort>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1 && arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ std::to_string(arguments.size()) + ", should be 1 or 2",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!WhichDataType(arguments[0].type).isString())
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2 && !WhichDataType(arguments[1].type).isUInt16())
|
||||
throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt16.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt16>();
|
||||
}
|
||||
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
UInt16 default_port = 0;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const auto * port_column = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[1]).column.get());
|
||||
if (!port_column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant UInt16", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
default_port = port_column->getValue<UInt16>();
|
||||
}
|
||||
|
||||
const ColumnPtr url_column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * url_strs = checkAndGetColumn<ColumnString>(url_column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<UInt16>::create();
|
||||
typename ColumnVector<UInt16>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(url_column->size());
|
||||
|
||||
vector(default_port, url_strs->getChars(), url_strs->getOffsets(), vec_res);
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
static void vector(UInt16 default_port, const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt16> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = extractPort(default_port, data, prev_offset, offsets[i] - prev_offset - 1);
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static UInt16 extractPort(UInt16 default_port, const ColumnString::Chars & buf, size_t offset, size_t size)
|
||||
{
|
||||
const char * p = reinterpret_cast<const char *>(&buf[0]) + offset;
|
||||
const char * end = p + size;
|
||||
|
||||
StringRef host = getURLHost(p, size);
|
||||
if (!host.size)
|
||||
return default_port;
|
||||
if (host.size == size)
|
||||
return default_port;
|
||||
|
||||
p = host.data + host.size;
|
||||
if (*p++ != ':')
|
||||
return default_port;
|
||||
|
||||
Int64 port = default_port;
|
||||
while (p < end)
|
||||
{
|
||||
if (*p == '/')
|
||||
break;
|
||||
if (!isNumericASCII(*p))
|
||||
return default_port;
|
||||
|
||||
port = (port * 10) + (*p - '0');
|
||||
if (port < 0 || port > UInt16(-1))
|
||||
return default_port;
|
||||
++p;
|
||||
}
|
||||
return port;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionPort(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionPort>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,7 @@ void registerFunctionDomain(FunctionFactory & factory);
|
||||
void registerFunctionDomainWithoutWWW(FunctionFactory & factory);
|
||||
void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory);
|
||||
void registerFunctionTopLevelDomain(FunctionFactory & factory);
|
||||
void registerFunctionPort(FunctionFactory & factory);
|
||||
void registerFunctionPath(FunctionFactory & factory);
|
||||
void registerFunctionPathFull(FunctionFactory & factory);
|
||||
void registerFunctionQueryString(FunctionFactory & factory);
|
||||
@ -33,6 +34,7 @@ void registerFunctionsURL(FunctionFactory & factory)
|
||||
registerFunctionDomainWithoutWWW(factory);
|
||||
registerFunctionFirstSignificantSubdomain(factory);
|
||||
registerFunctionTopLevelDomain(factory);
|
||||
registerFunctionPort(factory);
|
||||
registerFunctionPath(factory);
|
||||
registerFunctionPathFull(factory);
|
||||
registerFunctionQueryString(factory);
|
||||
|
24
tests/queries/0_stateless/01284_port.reference
Normal file
24
tests/queries/0_stateless/01284_port.reference
Normal file
@ -0,0 +1,24 @@
|
||||
ipv4
|
||||
0
|
||||
80
|
||||
80
|
||||
80
|
||||
80
|
||||
hostname
|
||||
0
|
||||
80
|
||||
80
|
||||
80
|
||||
80
|
||||
default-port
|
||||
80
|
||||
80
|
||||
ipv6
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
host-no-dot
|
||||
0
|
34
tests/queries/0_stateless/01284_port.sql
Normal file
34
tests/queries/0_stateless/01284_port.sql
Normal file
@ -0,0 +1,34 @@
|
||||
select 'ipv4';
|
||||
select port('http://127.0.0.1/');
|
||||
select port('http://127.0.0.1:80');
|
||||
select port('http://127.0.0.1:80/');
|
||||
select port('//127.0.0.1:80/');
|
||||
select port('127.0.0.1:80');
|
||||
select 'hostname';
|
||||
select port('http://foobar.com/');
|
||||
select port('http://foobar.com:80');
|
||||
select port('http://foobar.com:80/');
|
||||
select port('//foobar.com:80/');
|
||||
select port('foobar.com:80');
|
||||
|
||||
select 'default-port';
|
||||
select port('http://127.0.0.1/', toUInt16(80));
|
||||
select port('http://foobar.com/', toUInt16(80));
|
||||
|
||||
-- unsupported
|
||||
/* ILLEGAL_TYPE_OF_ARGUMENT */ select port(toFixedString('', 1)); -- { serverError 43; }
|
||||
/* ILLEGAL_TYPE_OF_ARGUMENT */ select port('', 1); -- { serverError 43; }
|
||||
/* NUMBER_OF_ARGUMENTS_DOESNT_MATCH */ select port('', 1, 1); -- { serverError 42; }
|
||||
|
||||
--
|
||||
-- Known limitations of domain() (getURLHost())
|
||||
--
|
||||
select 'ipv6';
|
||||
select port('http://[2001:db8::8a2e:370:7334]/');
|
||||
select port('http://[2001:db8::8a2e:370:7334]:80');
|
||||
select port('http://[2001:db8::8a2e:370:7334]:80/');
|
||||
select port('//[2001:db8::8a2e:370:7334]:80/');
|
||||
select port('[2001:db8::8a2e:370:7334]:80');
|
||||
select port('2001:db8::8a2e:370:7334:80');
|
||||
select 'host-no-dot';
|
||||
select port('//foobar:80/');
|
Loading…
Reference in New Issue
Block a user