mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge branch 'ClickHouse:master' into fix_in_operator_type_conversion
This commit is contained in:
commit
e3c7085ff7
@ -89,10 +89,6 @@ function configure()
|
|||||||
# since we run clickhouse from root
|
# since we run clickhouse from root
|
||||||
sudo chown root: /var/lib/clickhouse
|
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
|
local total_mem
|
||||||
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
|
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
|
||||||
total_mem=$(( total_mem*1024 )) # bytes
|
total_mem=$(( total_mem*1024 )) # bytes
|
||||||
|
@ -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_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_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_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}
|
## RowBinary {#rowbinary}
|
||||||
|
|
||||||
|
@ -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}
|
## Template format settings {#template-format-settings}
|
||||||
|
|
||||||
### format_template_resultset {#format_template_resultset}
|
### format_template_resultset {#format_template_resultset}
|
||||||
|
265
src/AggregateFunctions/AggregateFunctionGroupConcat.cpp
Normal file
265
src/AggregateFunctions/AggregateFunctionGroupConcat.cpp
Normal 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -19,6 +19,7 @@ void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factor
|
|||||||
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
|
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
|
||||||
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
|
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
|
||||||
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &);
|
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &);
|
||||||
|
void registerAggregateFunctionGroupConcat(AggregateFunctionFactory &);
|
||||||
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
|
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
|
||||||
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
|
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
|
||||||
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
|
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
|
||||||
@ -120,6 +121,7 @@ void registerAggregateFunctions()
|
|||||||
registerAggregateFunctionGroupUniqArray(factory);
|
registerAggregateFunctionGroupUniqArray(factory);
|
||||||
registerAggregateFunctionGroupArrayInsertAt(factory);
|
registerAggregateFunctionGroupArrayInsertAt(factory);
|
||||||
registerAggregateFunctionGroupArrayIntersect(factory);
|
registerAggregateFunctionGroupArrayIntersect(factory);
|
||||||
|
registerAggregateFunctionGroupConcat(factory);
|
||||||
registerAggregateFunctionsQuantile(factory);
|
registerAggregateFunctionsQuantile(factory);
|
||||||
registerAggregateFunctionsQuantileDeterministic(factory);
|
registerAggregateFunctionsQuantileDeterministic(factory);
|
||||||
registerAggregateFunctionsQuantileExact(factory);
|
registerAggregateFunctionsQuantileExact(factory);
|
||||||
|
@ -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, 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(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(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, 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(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) \
|
M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \
|
||||||
|
@ -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"},
|
{"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_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"},
|
{"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."},
|
{"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_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."},
|
{"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."},
|
||||||
|
@ -175,8 +175,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
|
|||||||
if (!mask_filled)
|
if (!mask_filled)
|
||||||
(*default_mask)[requested_key_index] = 1;
|
(*default_mask)[requested_key_index] = 1;
|
||||||
|
|
||||||
Field value{};
|
result_column->insertDefault();
|
||||||
result_column->insert(value);
|
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -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.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_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_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.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.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;
|
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;
|
||||||
|
@ -289,6 +289,8 @@ struct FormatSettings
|
|||||||
|
|
||||||
bool output_format_pretty_row_numbers = false;
|
bool output_format_pretty_row_numbers = false;
|
||||||
UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000;
|
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
|
enum class Charset : uint8_t
|
||||||
{
|
{
|
||||||
|
@ -116,6 +116,12 @@ struct GridSymbols
|
|||||||
const char * dash = "─";
|
const char * dash = "─";
|
||||||
const char * bold_bar = "┃";
|
const char * bold_bar = "┃";
|
||||||
const char * 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;
|
GridSymbols utf8_grid_symbols;
|
||||||
@ -182,47 +188,58 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
|||||||
Widths name_widths;
|
Widths name_widths;
|
||||||
calculateWidths(header, chunk, widths, max_widths, name_widths);
|
calculateWidths(header, chunk, widths, max_widths, name_widths);
|
||||||
|
|
||||||
const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ?
|
const GridSymbols & grid_symbols
|
||||||
utf8_grid_symbols :
|
= format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols;
|
||||||
ascii_grid_symbols;
|
|
||||||
|
|
||||||
/// Create separators
|
/// Create separators
|
||||||
WriteBufferFromOwnString top_separator;
|
WriteBufferFromOwnString top_separator;
|
||||||
WriteBufferFromOwnString middle_names_separator;
|
WriteBufferFromOwnString middle_names_separator;
|
||||||
WriteBufferFromOwnString middle_values_separator;
|
WriteBufferFromOwnString middle_values_separator;
|
||||||
WriteBufferFromOwnString bottom_separator;
|
WriteBufferFromOwnString bottom_separator;
|
||||||
|
WriteBufferFromOwnString footer_top_separator;
|
||||||
|
WriteBufferFromOwnString footer_bottom_separator;
|
||||||
|
|
||||||
top_separator << grid_symbols.bold_left_top_corner;
|
top_separator << grid_symbols.bold_left_top_corner;
|
||||||
middle_names_separator << grid_symbols.bold_left_separator;
|
middle_names_separator << grid_symbols.bold_left_separator;
|
||||||
middle_values_separator << grid_symbols.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)
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
{
|
{
|
||||||
if (i != 0)
|
if (i != 0)
|
||||||
{
|
{
|
||||||
top_separator << grid_symbols.bold_top_separator;
|
top_separator << grid_symbols.bold_top_separator;
|
||||||
middle_names_separator << grid_symbols.bold_middle_separator;
|
middle_names_separator << grid_symbols.bold_middle_separator;
|
||||||
middle_values_separator << grid_symbols.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)
|
for (size_t j = 0; j < max_widths[i] + 2; ++j)
|
||||||
{
|
{
|
||||||
top_separator << grid_symbols.bold_dash;
|
top_separator << grid_symbols.bold_dash;
|
||||||
middle_names_separator << grid_symbols.bold_dash;
|
middle_names_separator << grid_symbols.bold_dash;
|
||||||
middle_values_separator << grid_symbols.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";
|
top_separator << grid_symbols.bold_right_top_corner << "\n";
|
||||||
middle_names_separator << grid_symbols.bold_right_separator << "\n";
|
middle_names_separator << grid_symbols.bold_right_separator << "\n";
|
||||||
middle_values_separator << grid_symbols.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 top_separator_s = top_separator.str();
|
||||||
std::string middle_names_separator_s = middle_names_separator.str();
|
std::string middle_names_separator_s = middle_names_separator.str();
|
||||||
std::string middle_values_separator_s = middle_values_separator.str();
|
std::string middle_values_separator_s = middle_values_separator.str();
|
||||||
std::string bottom_separator_s = bottom_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)
|
if (format_settings.pretty.output_format_pretty_row_numbers)
|
||||||
{
|
{
|
||||||
@ -239,43 +256,47 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Names
|
/// Names
|
||||||
writeCString(grid_symbols.bold_bar, out);
|
auto write_names = [&]() -> void
|
||||||
writeCString(" ", out);
|
|
||||||
for (size_t i = 0; i < num_columns; ++i)
|
|
||||||
{
|
{
|
||||||
if (i != 0)
|
writeCString(grid_symbols.bold_bar, out);
|
||||||
|
writeCString(" ", out);
|
||||||
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
{
|
{
|
||||||
writeCString(" ", out);
|
if (i != 0)
|
||||||
writeCString(grid_symbols.bold_bar, out);
|
{
|
||||||
writeCString(" ", out);
|
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);
|
||||||
}
|
}
|
||||||
|
writeCString(" ", out);
|
||||||
const auto & col = header.getByPosition(i);
|
writeCString(grid_symbols.bold_bar, out);
|
||||||
|
writeCString("\n", out);
|
||||||
if (color)
|
};
|
||||||
writeCString("\033[1m", out);
|
write_names();
|
||||||
|
|
||||||
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);
|
|
||||||
|
|
||||||
if (format_settings.pretty.output_format_pretty_row_numbers)
|
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)
|
if (j != 0)
|
||||||
writeCString(grid_symbols.bar, out);
|
writeCString(grid_symbols.bar, out);
|
||||||
const auto & type = *header.getByPosition(j).type;
|
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],
|
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);
|
writeCString(grid_symbols.bar, out);
|
||||||
@ -332,8 +359,33 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
|||||||
/// Write left blank
|
/// Write left blank
|
||||||
writeString(String(row_number_width, ' '), out);
|
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;
|
total_rows += num_rows;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -57,7 +57,8 @@ PrettyCompactBlockOutputFormat::PrettyCompactBlockOutputFormat(WriteBuffer & out
|
|||||||
void PrettyCompactBlockOutputFormat::writeHeader(
|
void PrettyCompactBlockOutputFormat::writeHeader(
|
||||||
const Block & block,
|
const Block & block,
|
||||||
const Widths & max_widths,
|
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)
|
if (format_settings.pretty.output_format_pretty_row_numbers)
|
||||||
{
|
{
|
||||||
@ -70,14 +71,20 @@ void PrettyCompactBlockOutputFormat::writeHeader(
|
|||||||
ascii_grid_symbols;
|
ascii_grid_symbols;
|
||||||
|
|
||||||
/// Names
|
/// 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);
|
writeCString(grid_symbols.dash, out);
|
||||||
for (size_t i = 0; i < max_widths.size(); ++i)
|
for (size_t i = 0; i < max_widths.size(); ++i)
|
||||||
{
|
{
|
||||||
if (i != 0)
|
if (i != 0)
|
||||||
{
|
{
|
||||||
writeCString(grid_symbols.dash, out);
|
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);
|
writeCString(grid_symbols.dash, out);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -107,7 +114,10 @@ void PrettyCompactBlockOutputFormat::writeHeader(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
writeCString(grid_symbols.dash, out);
|
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);
|
writeCString("\n", out);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -195,13 +205,19 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po
|
|||||||
Widths name_widths;
|
Widths name_widths;
|
||||||
calculateWidths(header, chunk, widths, max_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)
|
for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i)
|
||||||
writeRow(i, header, chunk, widths, max_widths);
|
writeRow(i, header, chunk, widths, 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)
|
||||||
writeBottom(max_widths);
|
{
|
||||||
|
writeHeader(header, max_widths, name_widths, true);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
writeBottom(max_widths);
|
||||||
|
}
|
||||||
|
|
||||||
total_rows += num_rows;
|
total_rows += num_rows;
|
||||||
}
|
}
|
||||||
|
@ -17,7 +17,7 @@ public:
|
|||||||
String getName() const override { return "PrettyCompactBlockOutputFormat"; }
|
String getName() const override { return "PrettyCompactBlockOutputFormat"; }
|
||||||
|
|
||||||
private:
|
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 writeBottom(const Widths & max_widths);
|
||||||
void writeRow(
|
void writeRow(
|
||||||
size_t row_num,
|
size_t row_num,
|
||||||
|
@ -36,39 +36,46 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
|
|||||||
if (format_settings.pretty.output_format_pretty_row_numbers)
|
if (format_settings.pretty.output_format_pretty_row_numbers)
|
||||||
writeString(String(row_number_width, ' '), out);
|
writeString(String(row_number_width, ' '), out);
|
||||||
/// Names
|
/// Names
|
||||||
for (size_t i = 0; i < num_columns; ++i)
|
auto write_names = [&](const bool is_footer) -> void
|
||||||
{
|
{
|
||||||
if (i != 0)
|
for (size_t i = 0; i < num_columns; ++i)
|
||||||
writeCString(" ", out);
|
|
||||||
else
|
|
||||||
writeChar(' ', out);
|
|
||||||
|
|
||||||
const ColumnWithTypeAndName & col = header.getByPosition(i);
|
|
||||||
|
|
||||||
if (col.type->shouldAlignRightInPrettyFormats())
|
|
||||||
{
|
{
|
||||||
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);
|
writeChar(' ', out);
|
||||||
|
|
||||||
if (color)
|
const ColumnWithTypeAndName & col = header.getByPosition(i);
|
||||||
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)
|
if (col.type->shouldAlignRightInPrettyFormats())
|
||||||
writeChar(' ', out);
|
{
|
||||||
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
if (!is_footer)
|
||||||
writeCString("\n\n", out);
|
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)
|
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(
|
writeValueWithPadding(
|
||||||
*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
|
*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
|
||||||
}
|
}
|
||||||
|
|
||||||
writeReadableNumberTip(chunk);
|
writeReadableNumberTip(chunk);
|
||||||
writeChar('\n', out);
|
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;
|
total_rows += num_rows;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -9,3 +9,5 @@
|
|||||||
01287_max_execution_speed
|
01287_max_execution_speed
|
||||||
# Check after ConstantNode refactoring
|
# Check after ConstantNode refactoring
|
||||||
02154_parser_backtracking
|
02154_parser_backtracking
|
||||||
|
02944_variant_as_common_type
|
||||||
|
02942_variant_cast
|
||||||
|
@ -512,6 +512,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried(
|
|||||||
), error
|
), error
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.mark.skip(reason="test is flaky, waiting ClickHouse/issues/64451")
|
||||||
def test_query_is_canceled_with_inf_retries(cluster, broken_s3):
|
def test_query_is_canceled_with_inf_retries(cluster, broken_s3):
|
||||||
node = cluster.instances["node_with_inf_s3_retries"]
|
node = cluster.instances["node_with_inf_s3_retries"]
|
||||||
|
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
SET output_format_pretty_color=1;
|
SET output_format_pretty_color=1;
|
||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT 1 FORMAT PrettySpace;
|
SELECT 1 FORMAT PrettySpace;
|
||||||
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
||||||
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SET output_format_pretty_color = 0;
|
SET output_format_pretty_color = 0;
|
||||||
SHOW SETTING output_format_pretty_color;
|
SHOW SETTING output_format_pretty_color;
|
||||||
|
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SET output_format_pretty_color = 1;
|
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;
|
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;
|
||||||
|
@ -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 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 PrettyCompact;
|
||||||
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;
|
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT
|
SELECT
|
||||||
s,
|
s,
|
||||||
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,
|
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,
|
||||||
|
@ -1,2 +1,3 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SET output_format_pretty_max_column_pad_width = 250;
|
SET output_format_pretty_max_column_pad_width = 250;
|
||||||
SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes;
|
SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT
|
SELECT
|
||||||
s,
|
s,
|
||||||
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,
|
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,
|
||||||
|
@ -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_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;
|
SELECT 'привет' AS x, 'мир' AS y FORMAT Pretty;
|
||||||
|
|
||||||
SET output_format_pretty_max_value_width = 5;
|
SET output_format_pretty_max_value_width = 5;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT 'parseDateTimeBestEffortUS';
|
SELECT 'parseDateTimeBestEffortUS';
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
SET output_format_pretty_color=1;
|
SET output_format_pretty_color=1;
|
||||||
SET output_format_pretty_row_numbers=0;
|
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 Pretty;
|
||||||
SELECT * FROM numbers(10) FORMAT PrettyCompact;
|
SELECT * FROM numbers(10) FORMAT PrettyCompact;
|
||||||
SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock;
|
SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
set output_format_pretty_display_footer_column_names=0;
|
||||||
set output_format_write_statistics=0;
|
set output_format_write_statistics=0;
|
||||||
|
|
||||||
select * from numbers(100) settings max_result_rows = 1; -- { serverError TOO_MANY_ROWS_OR_BYTES }
|
select * from numbers(100) settings max_result_rows = 1; -- { serverError TOO_MANY_ROWS_OR_BYTES }
|
||||||
|
@ -3,6 +3,6 @@
|
|||||||
'PrettySpaceNoEscapesMonoBlock'] -%}
|
'PrettySpaceNoEscapesMonoBlock'] -%}
|
||||||
|
|
||||||
select '{{ format }}';
|
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 -%}
|
{% endfor -%}
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT 'parseDateTime64BestEffortUS';
|
SELECT 'parseDateTime64BestEffortUS';
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SET output_format_pretty_color=1;
|
SET output_format_pretty_color=1;
|
||||||
SET read_in_order_two_level_merge_threshold=1000000;
|
SET read_in_order_two_level_merge_threshold=1000000;
|
||||||
|
|
||||||
|
@ -1,5 +1,4 @@
|
|||||||
set allow_experimental_variant_type=1;
|
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 NULL::Variant(String, UInt64);
|
||||||
select 42::UInt64::Variant(String, UInt64);
|
select 42::UInt64::Variant(String, UInt64);
|
||||||
|
@ -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 allow_experimental_variant_type=1;
|
||||||
set use_variant_as_common_type=1;
|
set use_variant_as_common_type=1;
|
||||||
|
|
||||||
|
@ -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 Pretty;
|
||||||
SELECT 1_000_000 as a FORMAT PrettyNoEscapes;
|
SELECT 1_000_000 as a FORMAT PrettyNoEscapes;
|
||||||
SELECT 1_000_000 as a FORMAT PrettyMonoBlock;
|
SELECT 1_000_000 as a FORMAT PrettyMonoBlock;
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SET output_format_pretty_row_numbers = 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;
|
SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1;
|
||||||
|
14
tests/queries/0_stateless/03156_group_concat.reference
Normal file
14
tests/queries/0_stateless/03156_group_concat.reference
Normal 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
|
40
tests/queries/0_stateless/03156_group_concat.sql
Normal file
40
tests/queries/0_stateless/03156_group_concat.sql
Normal 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;
|
@ -1,3 +1,4 @@
|
|||||||
|
SET output_format_pretty_display_footer_column_names=0;
|
||||||
SELECT 123456789 AS x FORMAT PrettyCompact;
|
SELECT 123456789 AS x FORMAT PrettyCompact;
|
||||||
SELECT toNullable(123456789) AS x FORMAT PrettyCompact;
|
SELECT toNullable(123456789) AS x FORMAT PrettyCompact;
|
||||||
SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact;
|
SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact;
|
||||||
|
@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
. "$CUR_DIR"/../shell_config.sh
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
# default output_format_pretty_max_rows is 10K
|
# 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
|
||||||
|
@ -0,0 +1,4 @@
|
|||||||
|
{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%}
|
||||||
|
{'xxx':56}
|
||||||
|
{56:'xxx'}
|
||||||
|
{% endfor -%}
|
@ -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
@ -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;
|
@ -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;
|
Loading…
Reference in New Issue
Block a user