mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Revert "Add function getHttpHeader"
This commit is contained in:
parent
d6a09a6efa
commit
6eff22568f
@ -68,45 +68,6 @@ WHERE macro = 'test';
|
||||
└───────┴──────────────┘
|
||||
```
|
||||
|
||||
## getHttpHeader
|
||||
Returns the value of specified http header.If there is no such header or the request method is not http, it will return empty string.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
getHttpHeader(name);
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `name` — Http header name .[String](../../sql-reference/data-types/string.md#string)
|
||||
|
||||
**Returned value**
|
||||
|
||||
Value of the specified header.
|
||||
Type:[String](../../sql-reference/data-types/string.md#string).
|
||||
|
||||
|
||||
When we use `clickhouse-client` to execute this function, we'll always get empty string, because client doesn't use http protocol.
|
||||
```sql
|
||||
SELECT getHttpHeader('test')
|
||||
```
|
||||
result:
|
||||
|
||||
```text
|
||||
┌─getHttpHeader('test')─┐
|
||||
│ │
|
||||
└───────────────────────┘
|
||||
```
|
||||
Try to use http request:
|
||||
```shell
|
||||
echo "select getHttpHeader('X-Clickhouse-User')" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @-
|
||||
|
||||
#result
|
||||
default
|
||||
```
|
||||
|
||||
|
||||
## FQDN
|
||||
|
||||
Returns the fully qualified domain name of the ClickHouse server.
|
||||
|
@ -1,103 +0,0 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include "Disks/DiskType.h"
|
||||
#include "Interpreters/Context_fwd.h"
|
||||
#include <Core/Field.h>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/** Get the value of parameter in http headers.
|
||||
* If there no such parameter or the method of request is not
|
||||
* http, the function will return empty string.
|
||||
*/
|
||||
class FunctionGetHttpHeader : public IFunction, WithContext
|
||||
{
|
||||
private:
|
||||
|
||||
public:
|
||||
explicit FunctionGetHttpHeader(ContextPtr context_): WithContext(context_) {}
|
||||
|
||||
static constexpr auto name = "getHttpHeader";
|
||||
|
||||
static FunctionPtr create(ContextPtr context_)
|
||||
{
|
||||
return std::make_shared<FunctionGetHttpHeader>(context_);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument of function {} must have String type", getName());
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & client_info = getContext()->getClientInfo();
|
||||
const auto & method = client_info.http_method;
|
||||
const auto & headers = client_info.headers;
|
||||
const IColumn * arg_column = arguments[0].column.get();
|
||||
const ColumnString * arg_string = checkAndGetColumn<ColumnString>(arg_column);
|
||||
|
||||
if (!arg_string)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must be constant String", getName());
|
||||
|
||||
if (method != ClientInfo::HTTPMethod::GET && method != ClientInfo::HTTPMethod::POST)
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
|
||||
auto result_column = ColumnString::create();
|
||||
|
||||
const String default_value;
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
auto header_name = arg_string->getDataAt(row).toString();
|
||||
|
||||
if (!headers.has(header_name))
|
||||
result_column->insertData(default_value.data(), default_value.size());
|
||||
|
||||
const String & value = headers[header_name];
|
||||
result_column->insertData(value.data(), value.size());
|
||||
}
|
||||
|
||||
return result_column;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(GetHttpHeader)
|
||||
{
|
||||
factory.registerFunction<FunctionGetHttpHeader>();
|
||||
}
|
||||
|
||||
}
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
@ -97,7 +96,6 @@ public:
|
||||
|
||||
/// For mysql and postgresql
|
||||
UInt64 connection_id = 0;
|
||||
Poco::Net::NameValueCollection headers;
|
||||
|
||||
/// Comma separated list of forwarded IP addresses (from X-Forwarded-For for HTTP interface).
|
||||
/// It's expected that proxy appends the forwarded address to the end of the list.
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <set>
|
||||
#include <optional>
|
||||
#include <memory>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
#include <Poco/UUID.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Common/Macros.h>
|
||||
@ -4218,15 +4217,12 @@ void Context::setClientConnectionId(uint32_t connection_id_)
|
||||
client_info.connection_id = connection_id_;
|
||||
}
|
||||
|
||||
void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers)
|
||||
void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
{
|
||||
client_info.http_method = http_method;
|
||||
client_info.http_user_agent = http_user_agent;
|
||||
client_info.http_referer = http_referer;
|
||||
need_recalculate_access = true;
|
||||
|
||||
if (!http_headers.empty())
|
||||
client_info.headers = http_headers;
|
||||
}
|
||||
|
||||
void Context::setForwardedFor(const String & forwarded_for)
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
#include <Core/Types.h>
|
||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
||||
|
||||
#include <base/types.h>
|
||||
@ -641,7 +639,7 @@ public:
|
||||
void setClientInterface(ClientInfo::Interface interface);
|
||||
void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
|
||||
void setClientConnectionId(uint32_t connection_id);
|
||||
void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers = {});
|
||||
void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer);
|
||||
void setForwardedFor(const String & forwarded_for);
|
||||
void setQueryKind(ClientInfo::QueryKind query_kind);
|
||||
void setQueryKindInitial();
|
||||
|
@ -15,7 +15,6 @@
|
||||
#include <Interpreters/Cluster.h>
|
||||
|
||||
#include <magic_enum.hpp>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <condition_variable>
|
||||
@ -432,18 +431,17 @@ void Session::setClientConnectionId(uint32_t connection_id)
|
||||
prepared_client_info->connection_id = connection_id;
|
||||
}
|
||||
|
||||
void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers)
|
||||
void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
{
|
||||
if (session_context)
|
||||
{
|
||||
session_context->setHttpClientInfo(http_method, http_user_agent, http_referer, http_headers);
|
||||
session_context->setHttpClientInfo(http_method, http_user_agent, http_referer);
|
||||
}
|
||||
else
|
||||
{
|
||||
prepared_client_info->http_method = http_method;
|
||||
prepared_client_info->http_user_agent = http_user_agent;
|
||||
prepared_client_info->http_referer = http_referer;
|
||||
prepared_client_info->headers = http_headers;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/SessionTracker.h>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
@ -65,7 +64,7 @@ public:
|
||||
void setClientInterface(ClientInfo::Interface interface);
|
||||
void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
|
||||
void setClientConnectionId(uint32_t connection_id);
|
||||
void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers = {});
|
||||
void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer);
|
||||
void setForwardedFor(const String & forwarded_for);
|
||||
void setQuotaClientKey(const String & quota_key);
|
||||
void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
|
||||
|
@ -42,7 +42,6 @@
|
||||
#include <Poco/Net/HTTPBasicCredentials.h>
|
||||
#include <Poco/Net/HTTPStream.h>
|
||||
#include <Poco/MemoryStream.h>
|
||||
#include <Poco/Net/NameValueCollection.h>
|
||||
#include <Poco/StreamCopier.h>
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
@ -503,7 +502,7 @@ bool HTTPHandler::authenticateUser(
|
||||
else if (request.getMethod() == HTTPServerRequest::HTTP_POST)
|
||||
http_method = ClientInfo::HTTPMethod::POST;
|
||||
|
||||
session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", ""), request);
|
||||
session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", ""));
|
||||
session->setForwardedFor(request.get("X-Forwarded-For", ""));
|
||||
session->setQuotaClientKey(quota_key);
|
||||
|
||||
|
@ -320,7 +320,6 @@ geoDistance
|
||||
geohashDecode
|
||||
geohashEncode
|
||||
geohashesInBox
|
||||
getHttpHeader
|
||||
getMacro
|
||||
getOSKernelVersion
|
||||
getServerPort
|
||||
|
@ -1,13 +0,0 @@
|
||||
default
|
||||
default value1 value2
|
||||
default value1 value2
|
||||
default value1 value2
|
||||
1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7
|
||||
2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7
|
||||
3
|
||||
value_from_query_1 value_from_query_2 value_from_query_3 1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7
|
||||
value_from_query_1 value_from_query_2 value_from_query_3 2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7
|
||||
value_from_query_1 value_from_query_2 value_from_query_3 3
|
||||
default
|
||||
http_value1
|
||||
http_value2
|
@ -1,70 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
echo "SELECT getHttpHeader('X-Clickhouse-User')" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @-
|
||||
|
||||
echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \
|
||||
-H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @-
|
||||
|
||||
echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \
|
||||
-H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @-
|
||||
|
||||
echo "SELECT getHttpHeader('X-' || 'Clickhouse' || '-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \
|
||||
-H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @-
|
||||
|
||||
db_name=db_$(echo $RANDOM |md5sum |cut -c 1-4)
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS ${db_name};"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE ${db_name}.02884_get_http_header
|
||||
(id UInt32,
|
||||
http_user String DEFAULT getHttpHeader('X-Clickhouse-User'),
|
||||
http_key1 String DEFAULT getHttpHeader('http_header_key1'),
|
||||
http_key2 String DEFAULT getHttpHeader('http_header_key2'),
|
||||
http_key3 String DEFAULT getHttpHeader('http_header_key3'),
|
||||
http_key4 String DEFAULT getHttpHeader('http_header_key4'),
|
||||
http_key5 String DEFAULT getHttpHeader('http_header_key5'),
|
||||
http_key6 String DEFAULT getHttpHeader('http_header_key6'),
|
||||
http_key7 String DEFAULT getHttpHeader('http_header_key7')
|
||||
)
|
||||
Engine=MergeTree()
|
||||
ORDER BY id"
|
||||
|
||||
#Insert data via http request
|
||||
echo "INSERT INTO ${db_name}.02884_get_http_header (id) values (1)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \
|
||||
-H 'http_header_key1: row1_value1'\
|
||||
-H 'http_header_key2: row1_value2'\
|
||||
-H 'http_header_key3: row1_value3'\
|
||||
-H 'http_header_key4: row1_value4'\
|
||||
-H 'http_header_key5: row1_value5'\
|
||||
-H 'http_header_key6: row1_value6'\
|
||||
-H 'http_header_key7: row1_value7' 'http://localhost:8123/' -d @-
|
||||
|
||||
echo "INSERT INTO ${db_name}.02884_get_http_header (id) values (2)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \
|
||||
-H 'http_header_key1: row2_value1'\
|
||||
-H 'http_header_key2: row2_value2'\
|
||||
-H 'http_header_key3: row2_value3'\
|
||||
-H 'http_header_key4: row2_value4'\
|
||||
-H 'http_header_key5: row2_value5'\
|
||||
-H 'http_header_key6: row2_value6'\
|
||||
-H 'http_header_key7: row2_value7' 'http://localhost:8123/' -d @-
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM ${db_name}.02884_get_http_header ORDER BY id;"
|
||||
#Insert data via tcp client
|
||||
$CLICKHOUSE_CLIENT --param_db="$db_name" -q "INSERT INTO ${db_name}.02884_get_http_header (id) values (3)"
|
||||
$CLICKHOUSE_CLIENT --param_db="$db_name" -q "SELECT * FROM ${db_name}.02884_get_http_header where id = 3"
|
||||
echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2'), getHttpHeader('key_from_query_3'), * FROM ${db_name}.02884_get_http_header ORDER BY id" | curl -s -H 'X-Clickhouse-User: default' \
|
||||
-H 'X-ClickHouse-Key: ' -H 'key_from_query_1: value_from_query_1' -H 'key_from_query_2: value_from_query_2' -H 'key_from_query_3: value_from_query_3' 'http://localhost:8123/' -d @-
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${db_name}.02884_get_http_header"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS ${db_name}.02884_header_from_table (header_name String) Engine=Memory"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO ${db_name}.02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')"
|
||||
|
||||
echo "SELECT getHttpHeader(header_name) as value from (select * FROM ${db_name}.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \
|
||||
-H 'X-ClickHouse-Key: ' -H 'http_key1: http_value1' -H 'http_key2: http_value2' 'http://localhost:8123/' -d @-
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE ${db_name}"
|
@ -1521,7 +1521,6 @@ geohashesInBox
|
||||
geoip
|
||||
geospatial
|
||||
getMacro
|
||||
getHttpHeader
|
||||
getOSKernelVersion
|
||||
getServerPort
|
||||
getSetting
|
||||
|
Loading…
Reference in New Issue
Block a user