Merge remote-tracking branch 'rschu1ze/master' into reenable-session-caching

This commit is contained in:
Robert Schulze 2024-06-17 20:51:11 +00:00
commit cdcfcb99fc
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
79 changed files with 3418 additions and 216 deletions

2
contrib/aws-crt-cpp vendored

@ -1 +1 @@
Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0
Subproject commit 0217761556a7ba7ec537fe933d0ab1159096746e

View File

@ -89,10 +89,6 @@ function configure()
# since we run clickhouse from root
sudo chown root: /var/lib/clickhouse
# Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM).
echo "<clickhouse><asynchronous_metrics_update_period_s>1</asynchronous_metrics_update_period_s></clickhouse>" \
> /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml
local total_mem
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
total_mem=$(( total_mem*1024 )) # bytes

View File

@ -1490,6 +1490,8 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000
- [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`.
- [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`.
- [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`.
- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if table contains many rows. Default value - `true`.
- [output_format_pretty_display_footer_column_names_min_rows](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names_min_rows) - Sets the minimum number of rows for which a footer will be displayed if [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) is enabled. Default value - 50.
## RowBinary {#rowbinary}

View File

@ -508,7 +508,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `http_response_headers`, `response_content`, `query`, `query_param_name`.
`type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static).
- `query` — use with `predefined_query_handler` type, executes query when the handler is called.
@ -519,6 +519,8 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `http_response_headers` — use with any type, response headers map. Could be used to set content type as well.
- `response_content` — use with `static` type, response content sent to client, when using the prefix file:// or config://, find the content from the file or configuration sends to client.
Next are the configuration methods for different `type`.
@ -616,6 +618,33 @@ Return a message.
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
<defaults/>
</http_handlers>
```
`http_response_headers` could be used to set content type instead of `content_type`.
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/hi</url>
<handler>
<type>static</type>
<status>402</status>
<http_response_headers>
<Content-Type>text/html; charset=UTF-8</Content-Type>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
@ -696,6 +725,9 @@ Find the content from the file send to client.
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</rule>
@ -706,6 +738,9 @@ Find the content from the file send to client.
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</rule>

View File

@ -1706,6 +1706,43 @@ Result:
└────────────┘
```
## output_format_pretty_display_footer_column_names
Display column names in the footer if there are many table rows.
Possible values:
- 0 — No column names are displayed in the footer.
- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows) (50 by default).
Default value: `1`.
**Example**
Query:
```sql
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 1000);
```
Result:
```response
┌─number─┬─toTypeName(number)─┐
1. │ 0 │ UInt64 │
2. │ 1 │ UInt64 │
3. │ 2 │ UInt64 │
...
999. │ 998 │ UInt64 │
1000. │ 999 │ UInt64 │
└─number─┴─toTypeName(number)─┘
```
## output_format_pretty_display_footer_column_names_min_rows
Sets the minimum number of rows for which a footer with column names will be displayed if setting [output_format_pretty_display_footer_column_names](#output_format_pretty_display_footer_column_names) is enabled.
Default value: `50`.
## Template format settings {#template-format-settings}
### format_template_resultset {#format_template_resultset}

View File

@ -414,6 +414,8 @@ $ curl -v 'http://localhost:8123/predefined_query'
- `content_type` — используется со всеми типами, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `http_response_headers` — используется со всеми типами чтобы добавить кастомные хедеры в ответ. Может использоваться в том числе для задания хедера `Content-Type` вместо `content_type`.
- `response_content` — используется с типом`static`, содержимое ответа, отправленное клиенту, при использовании префикса file:// or config://, находит содержимое из файла или конфигурации, отправленного клиенту.
Далее приведены методы настройки для различных типов.
@ -509,6 +511,33 @@ max_final_threads 2
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
<defaults/>
</http_handlers>
```
`http_response_headers` так же может использоваться для определения `Content-Type` вместо `content_type`.
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/hi</url>
<handler>
<type>static</type>
<status>402</status>
<http_response_headers>
<Content-Type>text/html; charset=UTF-8</Content-Type>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
@ -589,6 +618,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</rule>
@ -599,6 +631,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</rule>

View File

@ -31,6 +31,7 @@ namespace DB
{
namespace ErrorCodes
{
extern const int AUTHENTICATION_FAILED;
extern const int SUPPORT_IS_DISABLED;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
@ -90,8 +91,10 @@ bool AuthenticationData::Util::checkPasswordBcrypt(std::string_view password [[m
{
#if USE_BCRYPT
int ret = bcrypt_checkpw(password.data(), reinterpret_cast<const char *>(password_bcrypt.data()));
/// Before 24.6 we didn't validate hashes on creation, so it could be that the stored hash is invalid
/// and it could not be decoded by the library
if (ret == -1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "BCrypt library failed: bcrypt_checkpw returned {}", ret);
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Internal failure decoding Bcrypt hash");
return (ret == 0);
#else
throw Exception(
@ -230,6 +233,17 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Password hash for the 'BCRYPT_PASSWORD' authentication type has length {} "
"but must be 59 or 60 bytes.", hash.size());
auto resized = hash;
resized.resize(64);
#if USE_BCRYPT
/// Verify that it is a valid hash
int ret = bcrypt_checkpw("", reinterpret_cast<const char *>(resized.data()));
if (ret == -1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'");
#endif
password_hash = hash;
password_hash.resize(64);
return;

View File

@ -0,0 +1,265 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Core/ServerSettings.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <Interpreters/castColumn.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
namespace
{
struct GroupConcatDataBase
{
UInt64 data_size = 0;
UInt64 allocated_size = 0;
char * data = nullptr;
void checkAndUpdateSize(UInt64 add, Arena * arena)
{
if (data_size + add >= allocated_size)
{
auto old_size = allocated_size;
allocated_size = std::max(2 * allocated_size, data_size + add);
data = arena->realloc(data, old_size, allocated_size);
}
}
void insertChar(const char * str, UInt64 str_size, Arena * arena)
{
checkAndUpdateSize(str_size, arena);
memcpy(data + data_size, str, str_size);
data_size += str_size;
}
};
struct GroupConcatData : public GroupConcatDataBase
{
using Offset = UInt64;
using Allocator = MixedAlignedArenaAllocator<alignof(Offset), 4096>;
using Offsets = PODArray<Offset, 32, Allocator>;
/// offset[i * 2] - beginning of the i-th row, offset[i * 2 + 1] - end of the i-th row
Offsets offsets;
UInt64 num_rows = 0;
UInt64 getSize(size_t i) const { return offsets[i * 2 + 1] - offsets[i * 2]; }
UInt64 getString(size_t i) const { return offsets[i * 2]; }
void insert(const IColumn * column, const SerializationPtr & serialization, size_t row_num, Arena * arena)
{
WriteBufferFromOwnString buff;
serialization->serializeText(*column, row_num, buff, {});
auto string = buff.stringView();
checkAndUpdateSize(string.size(), arena);
memcpy(data + data_size, string.data(), string.size());
offsets.push_back(data_size, arena);
data_size += string.size();
offsets.push_back(data_size, arena);
num_rows++;
}
};
template <bool has_limit>
class GroupConcatImpl final
: public IAggregateFunctionDataHelper<GroupConcatData, GroupConcatImpl<has_limit>>
{
static constexpr auto name = "groupConcat";
SerializationPtr serialization;
UInt64 limit;
const String delimiter;
public:
GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_)
: IAggregateFunctionDataHelper<GroupConcatData, GroupConcatImpl<has_limit>>(
{data_type_}, parameters_, std::make_shared<DataTypeString>())
, serialization(this->argument_types[0]->getDefaultSerialization())
, limit(limit_)
, delimiter(delimiter_)
{
}
String getName() const override { return name; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & cur_data = this->data(place);
if constexpr (has_limit)
if (cur_data.num_rows >= limit)
return;
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.insert(columns[0], serialization, row_num, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_data = this->data(place);
auto & rhs_data = this->data(rhs);
if (rhs_data.data_size == 0)
return;
if constexpr (has_limit)
{
UInt64 new_elems_count = std::min(rhs_data.num_rows, limit - cur_data.num_rows);
for (UInt64 i = 0; i < new_elems_count; ++i)
{
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.offsets.push_back(cur_data.data_size, arena);
cur_data.insertChar(rhs_data.data + rhs_data.getString(i), rhs_data.getSize(i), arena);
cur_data.num_rows++;
cur_data.offsets.push_back(cur_data.data_size, arena);
}
}
else
{
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.insertChar(rhs_data.data, rhs_data.data_size, arena);
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
auto & cur_data = this->data(place);
writeVarUInt(cur_data.data_size, buf);
writeVarUInt(cur_data.allocated_size, buf);
buf.write(cur_data.data, cur_data.data_size);
if constexpr (has_limit)
{
writeVarUInt(cur_data.num_rows, buf);
for (const auto & offset : cur_data.offsets)
writeVarUInt(offset, buf);
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
auto & cur_data = this->data(place);
readVarUInt(cur_data.data_size, buf);
readVarUInt(cur_data.allocated_size, buf);
buf.readStrict(cur_data.data, cur_data.data_size);
if constexpr (has_limit)
{
readVarUInt(cur_data.num_rows, buf);
cur_data.offsets.resize_exact(cur_data.num_rows * 2, arena);
for (auto & offset : cur_data.offsets)
readVarUInt(offset, buf);
}
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & cur_data = this->data(place);
if (cur_data.data_size == 0)
{
auto column_nullable = IColumn::mutate(makeNullable(to.getPtr()));
column_nullable->insertDefault();
return;
}
auto & column_string = assert_cast<ColumnString &>(to);
column_string.insertData(cur_data.data, cur_data.data_size);
}
bool allocatesMemoryInArena() const override { return true; }
};
AggregateFunctionPtr createAggregateFunctionGroupConcat(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);
bool has_limit = false;
UInt64 limit = 0;
String delimiter;
if (parameters.size() > 2)
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Incorrect number of parameters for aggregate function {}, should be 0, 1 or 2, got: {}", name, parameters.size());
if (!parameters.empty())
{
auto type = parameters[0].getType();
if (type != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name);
delimiter = parameters[0].get<String>();
}
if (parameters.size() == 2)
{
auto type = parameters[1].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name);
if ((type == Field::Types::Int64 && parameters[1].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].get<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get<Int64>());
has_limit = true;
limit = parameters[1].get<UInt64>();
}
if (has_limit)
return std::make_shared<GroupConcatImpl</* has_limit= */ true>>(argument_types[0], parameters, limit, delimiter);
else
return std::make_shared<GroupConcatImpl</* has_limit= */ false>>(argument_types[0], parameters, limit, delimiter);
}
}
void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties });
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive);
}
}

View File

@ -19,6 +19,7 @@ void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factor
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &);
void registerAggregateFunctionGroupConcat(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
@ -120,6 +121,7 @@ void registerAggregateFunctions()
registerAggregateFunctionGroupUniqArray(factory);
registerAggregateFunctionGroupArrayInsertAt(factory);
registerAggregateFunctionGroupArrayIntersect(factory);
registerAggregateFunctionGroupConcat(factory);
registerAggregateFunctionsQuantile(factory);
registerAggregateFunctionsQuantileDeterministic(factory);
registerAggregateFunctionsQuantileExact(factory);

View File

@ -112,9 +112,8 @@ namespace
return configuration.has(config_prefix + ".uri");
}
/*
* New syntax requires protocol prefix "<http> or <https>"
* */
/* New syntax requires protocol prefix "<http> or <https>"
*/
std::optional<std::string> getProtocolPrefix(
ProxyConfiguration::Protocol request_protocol,
const String & config_prefix,
@ -130,22 +129,18 @@ namespace
return protocol_prefix;
}
template <bool new_syntax>
std::optional<std::string> calculatePrefixBasedOnSettingsSyntax(
bool new_syntax,
ProxyConfiguration::Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
{
if (!configuration.has(config_prefix))
{
return std::nullopt;
}
if constexpr (new_syntax)
{
if (new_syntax)
return getProtocolPrefix(request_protocol, config_prefix, configuration);
}
return config_prefix;
}
@ -155,24 +150,21 @@ std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::
Protocol request_protocol,
const Poco::Util::AbstractConfiguration & configuration)
{
if (auto resolver = getFromSettings(request_protocol, "proxy", configuration))
{
if (auto resolver = getFromSettings(true, request_protocol, "proxy", configuration))
return resolver;
}
return std::make_shared<EnvironmentProxyConfigurationResolver>(
request_protocol,
isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration));
}
template <bool is_new_syntax>
std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::getFromSettings(
bool new_syntax,
Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
const Poco::Util::AbstractConfiguration & configuration)
{
auto prefix_opt = calculatePrefixBasedOnSettingsSyntax<is_new_syntax>(request_protocol, config_prefix, configuration);
auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(new_syntax, request_protocol, config_prefix, configuration);
if (!prefix_opt)
{
@ -195,20 +187,17 @@ std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::
std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::getFromOldSettingsFormat(
Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
const Poco::Util::AbstractConfiguration & configuration)
{
/*
* First try to get it from settings only using the combination of config_prefix and configuration.
/* First try to get it from settings only using the combination of config_prefix and configuration.
* This logic exists for backward compatibility with old S3 storage specific proxy configuration.
* */
if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings<false>(request_protocol, config_prefix + ".proxy", configuration))
if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(false, request_protocol, config_prefix + ".proxy", configuration))
{
return resolver;
}
/*
* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings.
/* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings.
* Falls back to Environment resolver if no configuration is found.
* */
return ProxyConfigurationResolverProvider::get(request_protocol, configuration);

View File

@ -33,12 +33,11 @@ public:
);
private:
template <bool is_new_syntax = true>
static std::shared_ptr<ProxyConfigurationResolver> getFromSettings(
bool is_new_syntax,
Protocol protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
);
const Poco::Util::AbstractConfiguration & configuration);
};
}

View File

@ -85,9 +85,18 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
/// Write information about current server instance to the file.
WriteBufferFromFileDescriptor out(fd, 1024);
fill(out);
/// Finalize here to avoid throwing exceptions in destructor.
out.finalize();
try
{
fill(out);
/// Finalize here to avoid throwing exceptions in destructor.
out.finalize();
}
catch (...)
{
/// Finalize in case of exception to avoid throwing exceptions in destructor
out.finalize();
throw;
}
}
catch (...)
{

View File

@ -1144,7 +1144,9 @@ class IColumn;
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \
M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \
M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \
M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \

View File

@ -108,6 +108,8 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"},
{"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"},
{"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"},
{"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."},
{"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."},
{"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."},

View File

@ -146,7 +146,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams(
}
/// If we started to read a new column, reinitialize variant column in deserialization state.
if (!variant_element_state->variant || result_column->empty())
if (!variant_element_state->variant || mutable_column->empty())
{
variant_element_state->variant = mutable_column->cloneEmpty();

View File

@ -175,8 +175,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
if (!mask_filled)
(*default_mask)[requested_key_index] = 1;
Field value{};
result_column->insert(value);
result_column->insertDefault();
}
else
{

View File

@ -181,6 +181,8 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.pretty.highlight_digit_groups = settings.output_format_pretty_highlight_digit_groups;
format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers;
format_settings.pretty.output_format_pretty_single_large_number_tip_threshold = settings.output_format_pretty_single_large_number_tip_threshold;
format_settings.pretty.output_format_pretty_display_footer_column_names = settings.output_format_pretty_display_footer_column_names;
format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows = settings.output_format_pretty_display_footer_column_names_min_rows;
format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;

View File

@ -289,6 +289,8 @@ struct FormatSettings
bool output_format_pretty_row_numbers = false;
UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000;
UInt64 output_format_pretty_display_footer_column_names = 1;
UInt64 output_format_pretty_display_footer_column_names_min_rows = 50;
enum class Charset : uint8_t
{

View File

@ -221,7 +221,7 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
if (iter == http_header_entries.end())
{
http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING));
http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}{}", VERSION_STRING, VERSION_OFFICIAL));
}
if (!delay_initialization && use_external_buffer)

View File

@ -972,10 +972,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT
{
auto context = Context::getGlobalContextInstance();
chassert(context);
auto proxy_configuration_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::protocolFromString(protocol), context->getConfigRef());
auto proxy_configuration_resolver = ProxyConfigurationResolverProvider::get(ProxyConfiguration::protocolFromString(protocol), context->getConfigRef());
auto per_request_configuration = [=] () { return proxy_configuration_resolver->resolve(); };
auto error_report = [=] (const DB::ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); };
auto per_request_configuration = [=]{ return proxy_configuration_resolver->resolve(); };
auto error_report = [=](const ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); };
auto config = PocoHTTPClientConfiguration(
per_request_configuration,

View File

@ -1,4 +1,5 @@
#include <Poco/Timespan.h>
#include <Common/config_version.h>
#include "config.h"
#if USE_AWS_S3
@ -17,6 +18,7 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <IO/S3/ProviderType.h>
#include <Interpreters/Context.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/HttpResponse.h>
@ -29,6 +31,7 @@
#include <boost/algorithm/string.hpp>
static const int SUCCESS_RESPONSE_MIN = 200;
static const int SUCCESS_RESPONSE_MAX = 299;
@ -84,7 +87,7 @@ namespace DB::S3
{
PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
std::function<DB::ProxyConfiguration()> per_request_configuration_,
std::function<ProxyConfiguration()> per_request_configuration_,
const String & force_region_,
const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_,
@ -94,7 +97,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
bool s3_use_adaptive_timeouts_,
const ThrottlerPtr & get_request_throttler_,
const ThrottlerPtr & put_request_throttler_,
std::function<void(const DB::ProxyConfiguration &)> error_report_)
std::function<void(const ProxyConfiguration &)> error_report_)
: per_request_configuration(per_request_configuration_)
, force_region(force_region_)
, remote_host_filter(remote_host_filter_)
@ -107,6 +110,8 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
, s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_)
, error_report(error_report_)
{
/// This is used to identify configurations created by us.
userAgent = std::string(VERSION_FULL) + VERSION_OFFICIAL;
}
void PocoHTTPClientConfiguration::updateSchemeAndRegion()
@ -166,6 +171,17 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config
{
}
PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration)
: timeouts(ConnectionTimeouts()
.withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000))
.withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withTCPKeepAliveTimeout(Poco::Timespan(
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))),
remote_host_filter(Context::getGlobalContextInstance()->getRemoteHostFilter())
{
}
std::shared_ptr<Aws::Http::HttpResponse> PocoHTTPClient::MakeRequest(
const std::shared_ptr<Aws::Http::HttpRequest> & request,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,

View File

@ -38,7 +38,7 @@ class PocoHTTPClient;
struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
{
std::function<DB::ProxyConfiguration()> per_request_configuration;
std::function<ProxyConfiguration()> per_request_configuration;
String force_region;
const RemoteHostFilter & remote_host_filter;
unsigned int s3_max_redirects;
@ -54,13 +54,13 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT;
size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST;
std::function<void(const DB::ProxyConfiguration &)> error_report;
std::function<void(const ProxyConfiguration &)> error_report;
void updateSchemeAndRegion();
private:
PocoHTTPClientConfiguration(
std::function<DB::ProxyConfiguration()> per_request_configuration_,
std::function<ProxyConfiguration()> per_request_configuration_,
const String & force_region_,
const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_,
@ -70,8 +70,7 @@ private:
bool s3_use_adaptive_timeouts_,
const ThrottlerPtr & get_request_throttler_,
const ThrottlerPtr & put_request_throttler_,
std::function<void(const DB::ProxyConfiguration &)> error_report_
);
std::function<void(const ProxyConfiguration &)> error_report_);
/// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization.
friend ClientFactory;
@ -120,6 +119,7 @@ class PocoHTTPClient : public Aws::Http::HttpClient
{
public:
explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration);
explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration);
~PocoHTTPClient() override = default;
std::shared_ptr<Aws::Http::HttpResponse> MakeRequest(
@ -166,8 +166,8 @@ protected:
static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request);
void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const;
std::function<DB::ProxyConfiguration()> per_request_configuration;
std::function<void(const DB::ProxyConfiguration &)> error_report;
std::function<ProxyConfiguration()> per_request_configuration;
std::function<void(const ProxyConfiguration &)> error_report;
ConnectionTimeouts timeouts;
const RemoteHostFilter & remote_host_filter;
unsigned int s3_max_redirects;

View File

@ -15,7 +15,10 @@ namespace DB::S3
std::shared_ptr<Aws::Http::HttpClient>
PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const
{
return std::make_shared<PocoHTTPClient>(static_cast<const PocoHTTPClientConfiguration &>(client_configuration));
if (client_configuration.userAgent.starts_with("ClickHouse"))
return std::make_shared<PocoHTTPClient>(static_cast<const PocoHTTPClientConfiguration &>(client_configuration));
else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost.
return std::make_shared<PocoHTTPClient>(client_configuration);
}
std::shared_ptr<Aws::Http::HttpRequest> PocoHTTPClientFactory::CreateHttpRequest(

View File

@ -77,7 +77,15 @@ WriteBufferFromFile::~WriteBufferFromFile()
if (fd < 0)
return;
finalize();
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
int err = ::close(fd);
/// Everything except for EBADF should be ignored in dtor, since all of
/// others (EINTR/EIO/ENOSPC/EDQUOT) could be possible during writing to

View File

@ -105,7 +105,14 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor(
WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor()
{
finalize();
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void WriteBufferFromFileDescriptor::finalizeImpl()

View File

@ -116,6 +116,12 @@ struct GridSymbols
const char * dash = "";
const char * bold_bar = "";
const char * bar = "";
const char * bold_right_separator_footer = "";
const char * bold_left_separator_footer = "";
const char * bold_middle_separator_footer = "";
const char * bold_left_bottom_corner = "";
const char * bold_right_bottom_corner = "";
const char * bold_bottom_separator = "";
};
GridSymbols utf8_grid_symbols;
@ -182,47 +188,58 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ?
utf8_grid_symbols :
ascii_grid_symbols;
const GridSymbols & grid_symbols
= format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols;
/// Create separators
WriteBufferFromOwnString top_separator;
WriteBufferFromOwnString middle_names_separator;
WriteBufferFromOwnString middle_values_separator;
WriteBufferFromOwnString bottom_separator;
WriteBufferFromOwnString footer_top_separator;
WriteBufferFromOwnString footer_bottom_separator;
top_separator << grid_symbols.bold_left_top_corner;
middle_names_separator << grid_symbols.bold_left_separator;
top_separator << grid_symbols.bold_left_top_corner;
middle_names_separator << grid_symbols.bold_left_separator;
middle_values_separator << grid_symbols.left_separator;
bottom_separator << grid_symbols.left_bottom_corner;
bottom_separator << grid_symbols.left_bottom_corner;
footer_top_separator << grid_symbols.bold_left_separator_footer;
footer_bottom_separator << grid_symbols.bold_left_bottom_corner;
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
{
top_separator << grid_symbols.bold_top_separator;
middle_names_separator << grid_symbols.bold_middle_separator;
top_separator << grid_symbols.bold_top_separator;
middle_names_separator << grid_symbols.bold_middle_separator;
middle_values_separator << grid_symbols.middle_separator;
bottom_separator << grid_symbols.bottom_separator;
bottom_separator << grid_symbols.bottom_separator;
footer_top_separator << grid_symbols.bold_middle_separator_footer;
footer_bottom_separator << grid_symbols.bold_bottom_separator;
}
for (size_t j = 0; j < max_widths[i] + 2; ++j)
{
top_separator << grid_symbols.bold_dash;
middle_names_separator << grid_symbols.bold_dash;
top_separator << grid_symbols.bold_dash;
middle_names_separator << grid_symbols.bold_dash;
middle_values_separator << grid_symbols.dash;
bottom_separator << grid_symbols.dash;
bottom_separator << grid_symbols.dash;
footer_top_separator << grid_symbols.bold_dash;
footer_bottom_separator << grid_symbols.bold_dash;
}
}
top_separator << grid_symbols.bold_right_top_corner << "\n";
middle_names_separator << grid_symbols.bold_right_separator << "\n";
top_separator << grid_symbols.bold_right_top_corner << "\n";
middle_names_separator << grid_symbols.bold_right_separator << "\n";
middle_values_separator << grid_symbols.right_separator << "\n";
bottom_separator << grid_symbols.right_bottom_corner << "\n";
bottom_separator << grid_symbols.right_bottom_corner << "\n";
footer_top_separator << grid_symbols.bold_right_separator_footer << "\n";
footer_bottom_separator << grid_symbols.bold_right_bottom_corner << "\n";
std::string top_separator_s = top_separator.str();
std::string middle_names_separator_s = middle_names_separator.str();
std::string middle_values_separator_s = middle_values_separator.str();
std::string bottom_separator_s = bottom_separator.str();
std::string footer_top_separator_s = footer_top_separator.str();
std::string footer_bottom_separator_s = footer_bottom_separator.str();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -239,43 +256,47 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
}
/// Names
writeCString(grid_symbols.bold_bar, out);
writeCString(" ", out);
for (size_t i = 0; i < num_columns; ++i)
auto write_names = [&]() -> void
{
if (i != 0)
writeCString(grid_symbols.bold_bar, out);
writeCString(" ", out);
for (size_t i = 0; i < num_columns; ++i)
{
writeCString(" ", out);
writeCString(grid_symbols.bold_bar, out);
writeCString(" ", out);
if (i != 0)
{
writeCString(" ", out);
writeCString(grid_symbols.bold_bar, out);
writeCString(" ", out);
}
const auto & col = header.getByPosition(i);
if (color)
writeCString("\033[1m", out);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
writeString(col.name, out);
}
else
{
writeString(col.name, out);
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
}
if (color)
writeCString("\033[0m", out);
}
const auto & col = header.getByPosition(i);
if (color)
writeCString("\033[1m", out);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
writeString(col.name, out);
}
else
{
writeString(col.name, out);
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
}
if (color)
writeCString("\033[0m", out);
}
writeCString(" ", out);
writeCString(grid_symbols.bold_bar, out);
writeCString("\n", out);
writeCString(" ", out);
writeCString(grid_symbols.bold_bar, out);
writeCString("\n", out);
};
write_names();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -317,9 +338,15 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
if (j != 0)
writeCString(grid_symbols.bar, out);
const auto & type = *header.getByPosition(j).type;
writeValueWithPadding(*columns[j], *serializations[j], i,
writeValueWithPadding(
*columns[j],
*serializations[j],
i,
widths[j].empty() ? max_widths[j] : widths[j][i],
max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
max_widths[j],
cut_to_width,
type.shouldAlignRightInPrettyFormats(),
isNumber(type));
}
writeCString(grid_symbols.bar, out);
@ -332,8 +359,33 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
writeString(bottom_separator_s, out);
/// output column names in the footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
{
writeString(footer_top_separator_s, out);
if (format_settings.pretty.output_format_pretty_row_numbers)
{
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
/// output header names
write_names();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
writeString(footer_bottom_separator_s, out);
}
else
{
writeString(bottom_separator_s, out);
}
total_rows += num_rows;
}

View File

@ -57,7 +57,8 @@ PrettyCompactBlockOutputFormat::PrettyCompactBlockOutputFormat(WriteBuffer & out
void PrettyCompactBlockOutputFormat::writeHeader(
const Block & block,
const Widths & max_widths,
const Widths & name_widths)
const Widths & name_widths,
const bool write_footer)
{
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -70,14 +71,20 @@ void PrettyCompactBlockOutputFormat::writeHeader(
ascii_grid_symbols;
/// Names
writeCString(grid_symbols.left_top_corner, out);
if (write_footer)
writeCString(grid_symbols.left_bottom_corner, out);
else
writeCString(grid_symbols.left_top_corner, out);
writeCString(grid_symbols.dash, out);
for (size_t i = 0; i < max_widths.size(); ++i)
{
if (i != 0)
{
writeCString(grid_symbols.dash, out);
writeCString(grid_symbols.top_separator, out);
if (write_footer)
writeCString(grid_symbols.bottom_separator, out);
else
writeCString(grid_symbols.top_separator, out);
writeCString(grid_symbols.dash, out);
}
@ -107,7 +114,10 @@ void PrettyCompactBlockOutputFormat::writeHeader(
}
}
writeCString(grid_symbols.dash, out);
writeCString(grid_symbols.right_top_corner, out);
if (write_footer)
writeCString(grid_symbols.right_bottom_corner, out);
else
writeCString(grid_symbols.right_top_corner, out);
writeCString("\n", out);
}
@ -195,13 +205,19 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
writeHeader(header, max_widths, name_widths);
writeHeader(header, max_widths, name_widths, false);
for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i)
writeRow(i, header, chunk, widths, max_widths);
writeBottom(max_widths);
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
{
writeHeader(header, max_widths, name_widths, true);
}
else
{
writeBottom(max_widths);
}
total_rows += num_rows;
}

View File

@ -17,7 +17,7 @@ public:
String getName() const override { return "PrettyCompactBlockOutputFormat"; }
private:
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths);
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths, bool write_footer);
void writeBottom(const Widths & max_widths);
void writeRow(
size_t row_num,

View File

@ -36,39 +36,46 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
if (format_settings.pretty.output_format_pretty_row_numbers)
writeString(String(row_number_width, ' '), out);
/// Names
for (size_t i = 0; i < num_columns; ++i)
auto write_names = [&](const bool is_footer) -> void
{
if (i != 0)
writeCString(" ", out);
else
writeChar(' ', out);
const ColumnWithTypeAndName & col = header.getByPosition(i);
if (col.type->shouldAlignRightInPrettyFormats())
for (size_t i = 0; i < num_columns; ++i)
{
for (ssize_t k = 0; k < std::max(0z, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
if (i != 0)
writeCString(" ", out);
else
writeChar(' ', out);
if (color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (color)
writeCString("\033[0m", out);
}
else
{
if (color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (color)
writeCString("\033[0m", out);
const ColumnWithTypeAndName & col = header.getByPosition(i);
for (ssize_t k = 0; k < std::max(0z, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', out);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (ssize_t k = 0; k < std::max(0z, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', out);
if (color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (color)
writeCString("\033[0m", out);
}
else
{
if (color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (color)
writeCString("\033[0m", out);
for (ssize_t k = 0; k < std::max(0z, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', out);
}
}
}
writeCString("\n\n", out);
if (!is_footer)
writeCString("\n\n", out);
else
writeCString("\n", out);
};
write_names(false);
for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row)
{
@ -95,11 +102,19 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
writeValueWithPadding(
*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
}
writeReadableNumberTip(chunk);
writeChar('\n', out);
}
/// Write blank line between last row and footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
writeCString("\n", out);
/// Write left blank
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names)
writeString(String(row_number_width, ' '), out);
/// Write footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
write_names(true);
total_rows += num_rows;
}

View File

@ -30,7 +30,6 @@
#include <Common/scope_guard_safe.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <Common/re2.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatFactory.h>
@ -44,6 +43,7 @@
#include <Poco/Base64Decoder.h>
#include <Poco/Base64Encoder.h>
#include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/Net/HTTPMessage.h>
#include <Poco/Net/HTTPStream.h>
#include <Poco/MemoryStream.h>
#include <Poco/StreamCopier.h>
@ -53,7 +53,10 @@
#include <algorithm>
#include <chrono>
#include <memory>
#include <optional>
#include <sstream>
#include <unordered_map>
#include <utility>
#if USE_SSL
#include <Poco/Net/X509Certificate.h>
@ -338,11 +341,11 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
}
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_)
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_)
: server(server_)
, log(getLogger(name))
, default_settings(server.context()->getSettingsRef())
, content_type_override(content_type_override_)
, http_response_headers_override(http_response_headers_override_)
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
@ -670,8 +673,7 @@ void HTTPHandler::processQuery(
{
auto tmp_data = std::make_shared<TemporaryDataOnDisk>(server.context()->getTempDataOnDisk());
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr
{
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr {
return tmp_data->createRawStream();
};
@ -893,13 +895,14 @@ void HTTPHandler::processQuery(
customizeContext(request, context, *in_post_maybe_compressed);
in = has_external_data ? std::move(in_param) : std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed);
applyHTTPResponseHeaders(response, http_response_headers_override);
auto set_query_result = [&response, this] (const QueryResultDetails & details)
{
response.add("X-ClickHouse-Query-Id", details.query_id);
if (content_type_override)
response.setContentType(*content_type_override);
else if (details.content_type)
if (!(http_response_headers_override && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
&& details.content_type)
response.setContentType(*details.content_type);
if (details.format)
@ -1185,8 +1188,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
used_output.finalize();
}
DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_, const std::optional<String>& content_type_override_)
: HTTPHandler(server_, "DynamicQueryHandler", content_type_override_), param_name(param_name_)
DynamicQueryHandler::DynamicQueryHandler(
IServer & server_, const std::string & param_name_, const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_)
{
}
@ -1247,8 +1251,8 @@ PredefinedQueryHandler::PredefinedQueryHandler(
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const std::optional<String> & content_type_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", content_type_override_)
const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_)
, receive_params(receive_params_)
, predefined_query(predefined_query_)
, url_regex(url_regex_)
@ -1340,14 +1344,10 @@ HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
{
auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
std::optional<String> content_type_override;
if (config.has(config_prefix + ".handler.content_type"))
content_type_override = config.getString(config_prefix + ".handler.content_type");
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
auto creator = [&server, query_param_name, content_type_override] () -> std::unique_ptr<DynamicQueryHandler>
{
return std::make_unique<DynamicQueryHandler>(server, query_param_name, content_type_override);
};
auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr<DynamicQueryHandler>
{ return std::make_unique<DynamicQueryHandler>(server, query_param_name, http_response_headers_override); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -1402,9 +1402,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
headers_name_with_regex.emplace(std::make_pair(header_name, regex));
}
std::optional<String> content_type_override;
if (config.has(config_prefix + ".handler.content_type"))
content_type_override = config.getString(config_prefix + ".handler.content_type");
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;
@ -1424,12 +1422,12 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
predefined_query,
regex,
headers_name_with_regex,
content_type_override]
http_response_headers_override]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, regex,
headers_name_with_regex, content_type_override);
headers_name_with_regex, http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -1442,12 +1440,12 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
analyze_receive_params,
predefined_query,
headers_name_with_regex,
content_type_override]
http_response_headers_override]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, CompiledRegexPtr{},
headers_name_with_regex, content_type_override);
headers_name_with_regex, http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));

View File

@ -1,5 +1,8 @@
#pragma once
#include <optional>
#include <string>
#include <unordered_map>
#include <Core/Names.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/HTTPRequestHandler.h>
@ -10,6 +13,8 @@
#include <Compression/CompressedWriteBuffer.h>
#include <Common/re2.h>
#include "HTTPResponseHeaderWriter.h"
namespace CurrentMetrics
{
extern const Metric HTTPConnection;
@ -31,7 +36,7 @@ using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
class HTTPHandler : public HTTPRequestHandler
{
public:
HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_);
HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_);
~HTTPHandler() override;
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
@ -113,8 +118,8 @@ private:
/// See settings http_max_fields, http_max_field_name_size, http_max_field_value_size in HTMLForm.
const Settings & default_settings;
/// Overrides Content-Type provided by the format of the response.
std::optional<String> content_type_override;
/// Overrides for response headers.
HTTPResponseHeaderSetup http_response_headers_override;
// session is reset at the end of each request/response.
std::unique_ptr<Session> session;
@ -162,8 +167,12 @@ class DynamicQueryHandler : public HTTPHandler
{
private:
std::string param_name;
public:
explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query", const std::optional<String>& content_type_override_ = std::nullopt);
explicit DynamicQueryHandler(
IServer & server_,
const std::string & param_name_ = "query",
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
@ -177,11 +186,15 @@ private:
std::string predefined_query;
CompiledRegexPtr url_regex;
std::unordered_map<String, CompiledRegexPtr> header_name_with_capture_regex;
public:
PredefinedQueryHandler(
IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_
, const CompiledRegexPtr & url_regex_, const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_
, const std::optional<std::string> & content_type_override_);
IServer & server_,
const NameSet & receive_params_,
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
void customizeContext(HTTPServerRequest & request, ContextMutablePtr context, ReadBuffer & body) override;

View File

@ -74,7 +74,8 @@ static auto createPingHandlerFactory(IServer & server)
auto creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto ping_response_expression = "Ok.\n";
return std::make_unique<StaticRequestHandler>(server, ping_response_expression);
return std::make_unique<StaticRequestHandler>(
server, ping_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
return std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));
}
@ -214,7 +215,8 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS
auto root_creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto root_response_expression = "config://http_server_default_response";
return std::make_unique<StaticRequestHandler>(server, root_response_expression);
return std::make_unique<StaticRequestHandler>(
server, root_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
auto root_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(root_creator));
root_handler->attachStrictPath("/");

View File

@ -0,0 +1,69 @@
#include "HTTPResponseHeaderWriter.h"
#include <unordered_map>
#include <utility>
#include <Poco/Net/HTTPMessage.h>
namespace DB
{
std::unordered_map<String, String>
baseParseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override;
String http_response_headers_key = config_prefix + ".handler.http_response_headers";
String http_response_headers_key_prefix = http_response_headers_key + ".";
if (config.has(http_response_headers_key))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(http_response_headers_key, keys);
for (const auto & key : keys)
{
http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key);
}
}
if (config.has(config_prefix + ".handler.content_type"))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type");
return http_response_headers_override;
}
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (http_response_headers_override.empty())
return {};
return std::move(http_response_headers_override);
}
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (!http_response_headers_override.contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = default_content_type;
return http_response_headers_override;
}
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type)
{
return {{{Poco::Net::HTTPMessage::CONTENT_TYPE, default_content_type}}};
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup)
{
if (setup)
for (const auto & [header_name, header_value] : *setup)
response.set(header_name, header_value);
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup)
{
for (const auto & [header_name, header_value] : setup)
response.set(header_name, header_value);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <optional>
#include <string>
#include <unordered_map>
#include <base/types.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using HTTPResponseHeaderSetup = std::optional<std::unordered_map<String, String>>;
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type);
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup);
}

View File

@ -2,7 +2,7 @@
#include "IServer.h"
#include "HTTPHandlerFactory.h"
#include "HTTPHandlerRequestFilter.h"
#include "HTTPResponseHeaderWriter.h"
#include <IO/HTTPCommon.h>
#include <IO/ReadBufferFromFile.h>
@ -14,6 +14,7 @@
#include <Common/Exception.h>
#include <unordered_map>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
@ -94,7 +95,7 @@ void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServer
try
{
response.setContentType(content_type);
applyHTTPResponseHeaders(response, http_response_headers_override);
if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1)
response.setChunkedTransferEncoding(true);
@ -155,8 +156,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out)
writeString(response_expression, out);
}
StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_)
: server(server_), status(status_), content_type(content_type_), response_expression(expression)
StaticRequestHandler::StaticRequestHandler(
IServer & server_, const String & expression, const std::unordered_map<String, String> & http_response_headers_override_, int status_)
: server(server_), status(status_), http_response_headers_override(http_response_headers_override_), response_expression(expression)
{
}
@ -166,12 +168,12 @@ HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
{
int status = config.getInt(config_prefix + ".handler.status", 200);
std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n");
std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
auto creator = [&server, response_content, status, response_content_type]() -> std::unique_ptr<StaticRequestHandler>
{
return std::make_unique<StaticRequestHandler>(server, response_content, status, response_content_type);
};
std::unordered_map<String, String> http_response_headers_override
= parseHTTPResponseHeaders(config, config_prefix, "text/plain; charset=UTF-8");
auto creator = [&server, http_response_headers_override, response_content, status]() -> std::unique_ptr<StaticRequestHandler>
{ return std::make_unique<StaticRequestHandler>(server, response_content, http_response_headers_override, status); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));

View File

@ -1,9 +1,9 @@
#pragma once
#include <unordered_map>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <base/types.h>
namespace DB
{
@ -17,15 +17,16 @@ private:
IServer & server;
int status;
String content_type;
/// Overrides for response headers.
std::unordered_map<String, String> http_response_headers_override;
String response_expression;
public:
StaticRequestHandler(
IServer & server,
const String & expression,
int status_ = 200,
const String & content_type_ = "text/html; charset=UTF-8");
const std::unordered_map<String, String> & http_response_headers_override_,
int status_ = 200);
void writeResponse(WriteBuffer & out);

View File

@ -132,11 +132,11 @@ void WriteBufferFromHDFS::sync()
}
void WriteBufferFromHDFS::finalizeImpl()
WriteBufferFromHDFS::~WriteBufferFromHDFS()
{
try
{
next();
finalize();
}
catch (...)
{
@ -144,11 +144,5 @@ void WriteBufferFromHDFS::finalizeImpl()
}
}
WriteBufferFromHDFS::~WriteBufferFromHDFS()
{
finalize();
}
}
#endif

View File

@ -38,8 +38,6 @@ public:
std::string getFileName() const override { return filename; }
private:
void finalizeImpl() override;
struct WriteBufferFromHDFSImpl;
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
const std::string filename;

View File

@ -83,7 +83,6 @@ void StorageObjectStorageSink::finalize()
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -91,6 +90,8 @@ void StorageObjectStorageSink::finalize()
release();
throw;
}
write_buf->finalize();
}
void StorageObjectStorageSink::release()

View File

@ -1823,7 +1823,6 @@ private:
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -1831,12 +1830,14 @@ private:
release();
throw;
}
write_buf->finalize();
}
void release()
{
writer.reset();
write_buf->finalize();
write_buf.reset();
}
StorageMetadataPtr metadata_snapshot;

View File

@ -609,7 +609,6 @@ void StorageURLSink::finalize()
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -617,12 +616,14 @@ void StorageURLSink::finalize()
release();
throw;
}
write_buf->finalize();
}
void StorageURLSink::release()
{
writer.reset();
write_buf->finalize();
write_buf.reset();
}
class PartitionedStorageURLSink : public PartitionedSink

View File

@ -9,3 +9,5 @@
01287_max_execution_speed
# Check after ConstantNode refactoring
02154_parser_backtracking
02944_variant_as_common_type
02942_variant_cast

View File

@ -88,6 +88,11 @@ def test_dynamic_query_handler():
"application/whatever; charset=cp1337"
== res_custom_ct.headers["content-type"]
)
assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"]
assert (
"also works"
== res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
def test_predefined_query_handler():
@ -146,6 +151,10 @@ def test_predefined_query_handler():
)
assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content
assert "application/generic+one" == res2.headers["content-type"]
assert "it works" == res2.headers["X-Test-Http-Response-Headers-Works"]
assert (
"also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
cluster.instance.query(
"CREATE TABLE test_table (id UInt32, data String) Engine=TinyLog"
@ -212,6 +221,18 @@ def test_fixed_static_handler():
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).content
)
assert (
"it works"
== cluster.instance.http_request(
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).headers["X-Test-Http-Response-Headers-Works"]
)
assert (
"also works"
== cluster.instance.http_request(
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
def test_config_static_handler():

View File

@ -18,6 +18,10 @@
<type>dynamic_query_handler</type>
<query_param_name>get_dynamic_handler_query</query_param_name>
<content_type>application/whatever; charset=cp1337</content_type>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>
</http_handlers>

View File

@ -19,6 +19,10 @@
<type>predefined_query_handler</type>
<query>SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String}</query>
<content_type>application/generic+one</content_type>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>
<rule>

View File

@ -12,6 +12,10 @@
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>Test get static handler and fix content</response_content>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>

View File

@ -1,4 +1,5 @@
SET output_format_pretty_color=1;
SET output_format_pretty_display_footer_column_names=0;
SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color = 0;
SHOW SETTING output_format_pretty_color;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color = 1;
SELECT number AS hello, toString(number) AS world, (hello, world) AS tuple, nullIf(hello % 3, 0) AS sometimes_nulls FROM system.numbers LIMIT 10 SETTINGS max_block_size = 5 FORMAT Pretty;

View File

@ -1,4 +1,4 @@
SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0;
SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, output_format_pretty_display_footer_column_names=0;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT
s,
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,

View File

@ -1,2 +1,3 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_max_column_pad_width = 250;
SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT
s,
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,

View File

@ -9,14 +9,14 @@ create table data_01256 as system.numbers Engine=Memory();
select 'min';
create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1,
2, 100, /* time */
5, 100, /* time */
4, 100, /* rows */
1, 1e6 /* bytes */
);
insert into buffer_01256 select * from system.numbers limit 5;
select count() from data_01256;
-- sleep 2 (min time) + 1 (round up) + bias (1) = 4
select sleepEachRow(2) from numbers(2) FORMAT Null;
-- It is enough to ensure that the buffer will be flushed earlier then 2*min_time (10 sec)
select sleepEachRow(9) FORMAT Null SETTINGS function_sleep_max_microseconds_per_block=10e6;
select count() from data_01256;
drop table buffer_01256;

View File

@ -1,4 +1,5 @@
SET output_format_pretty_color = 1, output_format_pretty_max_value_width_apply_for_single_value = 1, output_format_pretty_row_numbers = 0;
SET output_format_pretty_display_footer_column_names=0;
SELECT 'привет' AS x, 'мир' AS y FORMAT Pretty;
SET output_format_pretty_max_value_width = 5;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 'parseDateTimeBestEffortUS';
SELECT

View File

@ -1,5 +1,6 @@
SET output_format_pretty_color=1;
SET output_format_pretty_row_numbers=0;
SET output_format_pretty_display_footer_column_names=0;
SELECT * FROM numbers(10) FORMAT Pretty;
SELECT * FROM numbers(10) FORMAT PrettyCompact;
SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock;

View File

@ -1,3 +1,4 @@
set output_format_pretty_display_footer_column_names=0;
set output_format_write_statistics=0;
select * from numbers(100) settings max_result_rows = 1; -- { serverError TOO_MANY_ROWS_OR_BYTES }

View File

@ -3,6 +3,6 @@
'PrettySpaceNoEscapesMonoBlock'] -%}
select '{{ format }}';
select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1 format {{ format }};
select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1, output_format_pretty_display_footer_column_names=0 format {{ format }};
{% endfor -%}

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 'parseDateTime64BestEffortUS';
SELECT

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color=1;
SET read_in_order_two_level_merge_threshold=1000000;

View File

@ -1,5 +1,4 @@
set allow_experimental_variant_type=1;
set allow_experimental_analyzer=0; -- It's currently doesn't work with analyzer because of the way it works with constants, but it will be refactored and fixed in future
select NULL::Variant(String, UInt64);
select 42::UInt64::Variant(String, UInt64);

View File

@ -1,5 +1,3 @@
set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. It wil be fixed after refactoring around constants in analyzer.
set allow_experimental_variant_type=1;
set use_variant_as_common_type=1;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 1_000_000 as a FORMAT Pretty;
SELECT 1_000_000 as a FORMAT PrettyNoEscapes;
SELECT 1_000_000 as a FORMAT PrettyMonoBlock;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_row_numbers = 0;
SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1;

View File

@ -0,0 +1,14 @@
0 95 abc [1,2,3]
1 \N a [993,986,979,972]
2 123 makson95 []
95123
abcamakson95
[1,2,3][993,986,979,972][]
95,123
abc,a,makson95
[1,2,3],[993,986,979,972]
\N
951239512395123
abc,a,makson95,abc,a,makson95,abc,a,makson95
[1,2,3][993,986,979,972][][1,2,3][993,986,979,972][][1,2,3][993,986,979,972][]
488890

View File

@ -0,0 +1,40 @@
DROP TABLE IF EXISTS test_groupConcat;
CREATE TABLE test_groupConcat
(
id UInt64,
p_int Int32 NULL,
p_string String,
p_array Array(Int32)
) ENGINE = MergeTree ORDER BY id;
SET max_insert_threads = 1, max_threads = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
SELECT * FROM test_groupConcat;
SELECT groupConcat(p_int) FROM test_groupConcat;
SELECT groupConcat(p_string) FROM test_groupConcat;
SELECT groupConcat(p_array) FROM test_groupConcat;
SELECT groupConcat(',')(p_int) FROM test_groupConcat;
SELECT groupConcat(',')(p_string) FROM test_groupConcat;
SELECT groupConcat(',', 2)(p_array) FROM test_groupConcat;
SELECT groupConcat(p_int) FROM test_groupConcat WHERE id = 1;
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
SELECT groupConcat(p_int) FROM test_groupConcat;
SELECT groupConcat(',')(p_string) FROM test_groupConcat;
SELECT groupConcat(p_array) FROM test_groupConcat;
SELECT groupConcat(123)(number) FROM numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT groupConcat(',', '3')(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', 0)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', -1)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', 3, 3)(number) FROM numbers(10); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION }
SELECT length(groupConcat(number)) FROM numbers(100000);
DROP TABLE IF EXISTS test_groupConcat;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 123456789 AS x FORMAT PrettyCompact;
SELECT toNullable(123456789) AS x FORMAT PrettyCompact;
SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact;

View File

@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
# default output_format_pretty_max_rows is 10K
$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1" | wc -l
$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1, output_format_pretty_display_footer_column_names=0" | wc -l

View File

@ -0,0 +1,4 @@
{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%}
{'xxx':56}
{56:'xxx'}
{% endfor -%}

View File

@ -0,0 +1,31 @@
DROP TABLE IF EXISTS boom_filter_map_1;
DROP TABLE IF EXISTS boom_filter_map_2;
{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%}
CREATE TABLE boom_filter_map_1
(
`m` Map(String, {{ type }}),
INDEX index_models_value_bloom_filter mapValues(m) TYPE bloom_filter GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple();
CREATE TABLE boom_filter_map_2
(
`m` Map({{ type }}, String),
INDEX index_models_value_bloom_filter mapKeys(m) TYPE bloom_filter GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple();
INSERT INTO boom_filter_map_1 (m) values (map('xxx', 56));
INSERT INTO boom_filter_map_2 (m) values (map(56, 'xxx'));
SELECT m FROM boom_filter_map_1 WHERE (m['xxx']) = 56;
SELECT m FROM boom_filter_map_2 WHERE (m[56]) = 'xxx';
DROP TABLE IF EXISTS boom_filter_map_1;
DROP TABLE IF EXISTS boom_filter_map_2;
{% endfor -%}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,19 @@
-- https://github.com/ClickHouse/ClickHouse/issues/65035
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock;

View File

@ -0,0 +1,4 @@
1 2 3
4 5 6
7 8 9
0 0 0

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# Previous versions crashed in attempt to use this authentication method (regardless of whether it was able to authenticate):
AWS_CONTAINER_CREDENTIALS_FULL_URI=http://localhost:1338/latest/meta-data/container/security-credentials $CLICKHOUSE_LOCAL -q "select * from s3('http://localhost:11111/test/a.tsv')"

View File

@ -0,0 +1,37 @@
-- https://github.com/ClickHouse/ClickHouse/issues/65201
SET short_circuit_function_evaluation='enable';
DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes;
DROP TABLE IF EXISTS simple_key_simple_attributes_source_table;
CREATE TABLE simple_key_simple_attributes_source_table
(
id UInt64,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY direct_dictionary_simple_key_simple_attributes
(
`id` UInt64,
`value_first` String DEFAULT 'value_first_default',
`value_second` String DEFAULT 'value_second_default'
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table'))
LAYOUT(DIRECT());
SELECT
toUInt128(1),
dictGetOrDefault('direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString(toFixedString(toFixedString(toFixedString(materialize(toNullable('default')), 7), 7), toNullable(toNullable(toNullable(toUInt128(7))))))) AS value_second
FROM system.numbers LIMIT 255
FORMAT Null;
DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes;
DROP TABLE IF EXISTS simple_key_simple_attributes_source_table;

View File

@ -0,0 +1,3 @@
-- Tags: no-fasttest
DROP USER IF EXISTS 03172_user_invalid_bcrypt_hash;
CREATE USER 03172_user_invalid_bcrypt_hash IDENTIFIED WITH bcrypt_hash BY '012345678901234567890123456789012345678901234567890123456789'; -- { serverError BAD_ARGUMENTS }