Merge branch 'master' of github.com:ClickHouse/ClickHouse into fp16

This commit is contained in:
Alexey Milovidov 2024-11-11 00:40:39 +01:00
commit 73eba44238
48 changed files with 1913 additions and 153 deletions

2
contrib/SimSIMD vendored

@ -1 +1 @@
Subproject commit ee3c9c9c00b51645f62a1a9e99611b78c0052a21
Subproject commit fa60f1b8e3582c50978f0ae86c2ebb6c9af957f3

View File

@ -33,6 +33,21 @@ Then, generate the data. Parameter `-s` specifies the scale factor. For example,
./dbgen -s 100
```
Detailed table sizes with scale factor 100:
| Table | size (in rows) | size (compressed in ClickHouse) |
|----------|----------------|---------------------------------|
| nation | 25 | 2 kB |
| region | 5 | 1 kB |
| part | 20.000.000 | 895 MB |
| supplier | 1.000.000 | 75 MB |
| partsupp | 80.000.000 | 4.37 GB |
| customer | 15.000.000 | 1.19 GB |
| orders | 150.000.000 | 6.15 GB |
| lineitem | 600.00.00 | 26.69 GB |
(The table sizes in ClickHouse are taken from `system.tables.total_bytes` and based on below table definitions.
Now create tables in ClickHouse.
We stick as closely as possible to the rules of the TPC-H specification:
@ -151,10 +166,37 @@ clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO orders FORMAT
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO lineitem FORMAT CSV" < lineitem.tbl
```
The queries are generated by `./qgen -s <scaling_factor>`. Example queries for `s = 100`:
:::note
Instead of using tpch-kit and generating the tables by yourself, you can alternatively import the data from a public S3 bucket. Make sure
to create empty tables first using above `CREATE` statements.
```sql
-- Scaling factor 1
INSERT INTO nation SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/nation.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO region SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/region.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO part SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/part.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO supplier SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/supplier.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO partsupp SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/partsupp.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO customer SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/customer.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO orders SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/orders.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO lineitem SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/1/lineitem.tbl', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
-- Scaling factor 100
INSERT INTO nation SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/nation.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO region SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/region.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO part SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/part.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO supplier SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/supplier.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO partsupp SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/partsupp.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO customer SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/customer.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO orders SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/orders.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
INSERT INTO lineitem SELECT * FROM s3('https://clickhouse-datasets.s3.amazonaws.com/h/100/lineitem.tbl.gz', NOSIGN, CSV) SETTINGS format_csv_delimiter = '|', input_format_defaults_for_omitted_fields = 1, input_format_csv_empty_as_default = 1;
````
:::
## Queries
The queries are generated by `./qgen -s <scaling_factor>`. Example queries for `s = 100`:
**Correctness**
The result of the queries agrees with the official results unless mentioned otherwise. To verify, generate a TPC-H database with scale

View File

@ -22,7 +22,6 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/registerInterpreters.h>
#include <base/getFQDNOrHostName.h>
#include <Access/AccessControl.h>
#include <Common/PoolId.h>
#include <Common/Exception.h>
@ -31,7 +30,6 @@
#include <Common/ThreadStatus.h>
#include <Common/TLDListsHolder.h>
#include <Common/quoteString.h>
#include <Common/randomSeed.h>
#include <Common/ThreadPool.h>
#include <Common/CurrentMetrics.h>
#include <Loggers/OwnFormattingChannel.h>
@ -50,7 +48,6 @@
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
#include <boost/algorithm/string/replace.hpp>
#include <boost/program_options/options_description.hpp>
#include <base/argsToConfig.h>
#include <filesystem>
@ -71,9 +68,11 @@ namespace CurrentMetrics
namespace DB
{
namespace Setting
{
extern const SettingsBool allow_introspection_functions;
extern const SettingsBool implicit_select;
extern const SettingsLocalFSReadMethod storage_file_read_method;
}
@ -126,6 +125,7 @@ void applySettingsOverridesForLocal(ContextMutablePtr context)
settings[Setting::allow_introspection_functions] = true;
settings[Setting::storage_file_read_method] = LocalFSReadMethod::mmap;
settings[Setting::implicit_select] = true;
context->setSettings(settings);
}

View File

@ -51,7 +51,6 @@
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/JoinNode.h>
#include <Analyzer/UnionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Identifier.h>
@ -3023,9 +3022,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
argument_column.name = arguments_projection_names[function_argument_index];
/** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction
* where function argument types are initialized with empty array of lambda arguments size.
* where function argument types are initialized with empty arrays of lambda arguments size.
*/
if (const auto * lambda_node = function_argument->as<const LambdaNode>())
const auto * lambda_node = function_argument->as<const LambdaNode>();
if (lambda_node)
{
size_t lambda_arguments_size = lambda_node->getArguments().getNodes().size();
argument_column.type = std::make_shared<DataTypeFunction>(DataTypes(lambda_arguments_size, nullptr), nullptr);
@ -3497,15 +3497,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
else
function_base = function->build(argument_columns);
/// Do not constant fold get scalar functions
// bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" ||
// function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort";
/** If function is suitable for constant folding try to convert it to constant.
* Example: SELECT plus(1, 1);
* Result: SELECT 2;
*/
if (function_base->isSuitableForConstantFolding()) // && !disable_constant_folding)
if (function_base->isSuitableForConstantFolding())
{
auto result_type = function_base->getResultType();
auto executable_function = function_base->prepare(argument_columns);
@ -3514,7 +3510,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
if (all_arguments_constants)
{
size_t num_rows = function_arguments.empty() ? 0 : argument_columns.front().column->size();
size_t num_rows = 0;
if (!argument_columns.empty())
num_rows = argument_columns.front().column->size();
column = executable_function->execute(argument_columns, result_type, num_rows, true);
}
else

View File

@ -2679,7 +2679,10 @@ void ClientBase::runInteractive()
#if USE_REPLXX
replxx::Replxx::highlighter_callback_t highlight_callback{};
if (getClientConfiguration().getBool("highlight", true))
highlight_callback = highlight;
highlight_callback = [this](const String & query, std::vector<replxx::Replxx::Color> & colors)
{
highlight(query, colors, *client_context);
};
ReplxxLineReader lr(
*suggest,

View File

@ -5,6 +5,8 @@
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Common/UTF8Helpers.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <iostream>
@ -12,6 +14,11 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool implicit_select;
}
/// Should we celebrate a bit?
bool isNewYearMode()
{
@ -95,7 +102,7 @@ bool isChineseNewYearMode(const String & local_tz)
}
#if USE_REPLXX
void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors)
void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors, const Context & context)
{
using namespace replxx;
@ -135,7 +142,7 @@ void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors
/// Currently we highlight only the first query in the multi-query mode.
ParserQuery parser(end);
ParserQuery parser(end, false, context.getSettingsRef()[Setting::implicit_select]);
ASTPtr ast;
bool parse_res = false;

View File

@ -11,13 +11,15 @@
namespace DB
{
class Context;
/// Should we celebrate a bit?
bool isNewYearMode();
bool isChineseNewYearMode(const String & local_tz);
#if USE_REPLXX
void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors);
void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors, const Context & context);
#endif
}

View File

@ -72,6 +72,26 @@ ColumnPtr ColumnFunction::cut(size_t start, size_t length) const
return ColumnFunction::create(length, function, capture, is_short_circuit_argument, is_function_compiled);
}
Field ColumnFunction::operator[](size_t n) const
{
Field res;
get(n, res);
return res;
}
void ColumnFunction::get(size_t n, Field & res) const
{
const size_t tuple_size = captured_columns.size();
res = Tuple();
Tuple & res_tuple = res.safeGet<Tuple &>();
res_tuple.reserve(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
res_tuple.push_back((*captured_columns[i].column)[n]);
}
#if !defined(DEBUG_OR_SANITIZER_BUILD)
void ColumnFunction::insertFrom(const IColumn & src, size_t n)
#else

View File

@ -60,15 +60,9 @@ public:
void appendArguments(const ColumnsWithTypeAndName & columns);
ColumnWithTypeAndName reduce() const;
Field operator[](size_t) const override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
}
Field operator[](size_t n) const override;
void get(size_t, Field &) const override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName());
}
void get(size_t n, Field & res) const override;
StringRef getDataAt(size_t) const override
{

View File

@ -5708,6 +5708,8 @@ If enabled, MongoDB tables will return an error when a MongoDB query cannot be b
)", 0) \
DECLARE(Bool, implicit_select, false, R"(
Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query.
In `clickhouse-local` it is enabled by default and can be explicitly disabled.
)", 0) \
\
\

View File

@ -0,0 +1,107 @@
#include <Formats/PrettyFormatHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Processors/Chunk.h>
#include <Common/formatReadable.h>
static constexpr const char * GRAY_COLOR = "\033[90m";
static constexpr const char * UNDERSCORE = "\033[4m";
static constexpr const char * RESET_COLOR = "\033[0m";
namespace DB
{
void writeReadableNumberTipIfSingleValue(WriteBuffer & out, const Chunk & chunk, const FormatSettings & settings, bool color)
{
if (chunk.getNumRows() == 1 && chunk.getNumColumns() == 1)
writeReadableNumberTip(out, *chunk.getColumns()[0], 0, settings, color);
}
void writeReadableNumberTip(WriteBuffer & out, const IColumn & column, size_t row, const FormatSettings & settings, bool color)
{
if (column.isNullAt(row))
return;
auto value = column.getFloat64(row);
auto threshold = settings.pretty.output_format_pretty_single_large_number_tip_threshold;
if (threshold && isFinite(value) && abs(value) > threshold)
{
if (color)
writeCString(GRAY_COLOR, out);
writeCString(" -- ", out);
formatReadableQuantity(value, out, 2);
if (color)
writeCString(RESET_COLOR, out);
}
}
String highlightDigitGroups(String source)
{
if (source.size() <= 4)
return source;
bool is_regular_number = true;
size_t num_digits_before_decimal = 0;
for (auto c : source)
{
if (c == '-' || c == ' ')
continue;
if (c == '.')
break;
if (c >= '0' && c <= '9')
{
++num_digits_before_decimal;
}
else
{
is_regular_number = false;
break;
}
}
if (!is_regular_number || num_digits_before_decimal <= 4)
return source;
String result;
size_t size = source.size();
result.reserve(2 * size);
bool before_decimal = true;
size_t digit_num = 0;
for (size_t i = 0; i < size; ++i)
{
auto c = source[i];
if (before_decimal && c >= '0' && c <= '9')
{
++digit_num;
size_t offset = num_digits_before_decimal - digit_num;
if (offset && offset % 3 == 0)
{
result += UNDERSCORE;
result += c;
result += RESET_COLOR;
}
else
{
result += c;
}
}
else if (c == '.')
{
before_decimal = false;
result += c;
}
else
{
result += c;
}
}
return result;
}
}

View File

@ -0,0 +1,21 @@
#pragma once
#include <base/types.h>
namespace DB
{
class Chunk;
class IColumn;
class WriteBuffer;
struct FormatSettings;
/// Prints text describing the number in the form of: -- 12.34 million
void writeReadableNumberTip(WriteBuffer & out, const IColumn & column, size_t row, const FormatSettings & settings, bool color);
void writeReadableNumberTipIfSingleValue(WriteBuffer & out, const Chunk & chunk, const FormatSettings & settings, bool color);
/// Underscores digit groups related to thousands using terminal ANSI escape sequences.
String highlightDigitGroups(String source);
}

View File

@ -6,6 +6,7 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Columns/ColumnFunction.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypesNumber.h>
@ -112,6 +113,7 @@ public:
NamesAndTypesList lambda_arguments;
String return_name;
DataTypePtr return_type;
bool allow_constant_folding;
};
using CapturePtr = std::shared_ptr<Capture>;
@ -122,6 +124,7 @@ public:
String getName() const override { return "FunctionCapture"; }
bool useDefaultImplementationForNulls() const override { return false; }
/// It's possible if expression_actions contains function that don't use
/// default implementation for Nothing and one of captured columns can be Nothing
/// Example: SELECT arrayMap(x -> [x, arrayElement(y, 0)], []), [] as y
@ -148,7 +151,26 @@ public:
auto function = std::make_unique<FunctionExpression>(expression_actions, types, names,
capture->return_type, capture->return_name);
return ColumnFunction::create(input_rows_count, std::move(function), arguments);
/// If all the captured arguments are constant, let's also return ColumnConst (with ColumnFunction inside it).
/// Consequently, it allows to treat higher order functions with constant arrays and constant captured columns
/// as constant expressions.
/// Consequently, it allows its usage in contexts requiring constants, such as the right hand side of IN.
bool constant_folding = capture->allow_constant_folding
&& std::all_of(arguments.begin(), arguments.end(),
[](const auto & arg) { return arg.column->isConst(); });
if (constant_folding)
{
ColumnsWithTypeAndName arguments_resized = arguments;
for (auto & elem : arguments_resized)
elem.column = elem.column->cloneResized(1);
return ColumnConst::create(ColumnFunction::create(1, std::move(function), arguments_resized), input_rows_count);
}
else
{
return ColumnFunction::create(input_rows_count, std::move(function), arguments);
}
}
private:
@ -203,7 +225,8 @@ public:
const Names & captured_names,
const NamesAndTypesList & lambda_arguments,
const DataTypePtr & function_return_type,
const String & expression_return_name)
const String & expression_return_name,
bool allow_constant_folding)
: expression_actions(std::move(expression_actions_))
{
/// Check that expression does not contain unusual actions that will break columns structure.
@ -246,6 +269,7 @@ public:
.lambda_arguments = lambda_arguments,
.return_name = expression_return_name,
.return_type = function_return_type,
.allow_constant_folding = allow_constant_folding,
});
}

View File

@ -184,7 +184,7 @@ public:
/** If function isSuitableForConstantFolding then, this method will be called during query analysis
* if some arguments are constants. For example logical functions (AndFunction, OrFunction) can
* return they result based on some constant arguments.
* return the result based on some constant arguments.
* Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForNothing,
* useDefaultImplementationForConstants, useDefaultImplementationForLowCardinality are not applied.
*/

View File

@ -282,7 +282,9 @@ public:
if (!column_with_type_and_name.column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName());
const auto * column_function = typeid_cast<const ColumnFunction *>(column_with_type_and_name.column.get());
auto column_function_materialized = column_with_type_and_name.column->convertToFullColumnIfConst();
const auto * column_function = typeid_cast<const ColumnFunction *>(column_function_materialized.get());
if (!column_function)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName());

View File

@ -87,7 +87,9 @@ public:
if (!lambda_function_with_type_and_name.column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName());
const auto * lambda_function = typeid_cast<const ColumnFunction *>(lambda_function_with_type_and_name.column.get());
auto lambda_function_materialized = lambda_function_with_type_and_name.column->convertToFullColumnIfConst();
const auto * lambda_function = typeid_cast<const ColumnFunction *>(lambda_function_materialized.get());
if (!lambda_function)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName());

View File

@ -1308,7 +1308,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
String lambda_name = data.getUniqueName("__lambda");
auto function_capture = std::make_shared<FunctionCaptureOverloadResolver>(
lambda_actions, captured, lambda_arguments, result_type, result_name);
lambda_actions, captured, lambda_arguments, result_type, result_name, false);
data.addFunction(function_capture, captured, lambda_name);
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);

View File

@ -1467,7 +1467,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup);
if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings()[ServerSetting::ignore_empty_sql_security_in_create_view_query])
if (!create.sql_security && create.supportSQLSecurity() && (create.refresh_strategy || !getContext()->getServerSettings()[ServerSetting::ignore_empty_sql_security_in_create_view_query]))
create.sql_security = std::make_shared<ASTSQLSecurity>();
if (create.sql_security)

View File

@ -795,9 +795,9 @@ BlockIO InterpreterSystemQuery::execute()
case Type::WAIT_FAILPOINT:
{
getContext()->checkAccess(AccessType::SYSTEM_FAILPOINT);
LOG_TRACE(log, "waiting for failpoint {}", query.fail_point_name);
LOG_TRACE(log, "Waiting for failpoint {}", query.fail_point_name);
FailPointInjection::pauseFailPoint(query.fail_point_name);
LOG_TRACE(log, "finished failpoint {}", query.fail_point_name);
LOG_TRACE(log, "Finished waiting for failpoint {}", query.fail_point_name);
break;
}
case Type::RESET_COVERAGE:

View File

@ -804,7 +804,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
auto lambda_node_name = calculateActionNodeName(node, *planner_context);
auto function_capture = std::make_shared<FunctionCaptureOverloadResolver>(
lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name);
lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name, true);
// TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver.
const auto * actions_node = actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture);

View File

@ -1,5 +1,6 @@
#include <Processors/Formats/Impl/PrettyBlockOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/PrettyFormatHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
@ -352,7 +353,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
}
writeCString(grid_symbols.bar, out);
writeReadableNumberTip(chunk);
if (readable_number_tip)
writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color);
writeCString("\n", out);
}
@ -392,72 +394,6 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
}
static String highlightDigitGroups(String source)
{
if (source.size() <= 4)
return source;
bool is_regular_number = true;
size_t num_digits_before_decimal = 0;
for (auto c : source)
{
if (c == '-' || c == ' ')
continue;
if (c == '.')
break;
if (c >= '0' && c <= '9')
{
++num_digits_before_decimal;
}
else
{
is_regular_number = false;
break;
}
}
if (!is_regular_number || num_digits_before_decimal <= 4)
return source;
String result;
size_t size = source.size();
result.reserve(2 * size);
bool before_decimal = true;
size_t digit_num = 0;
for (size_t i = 0; i < size; ++i)
{
auto c = source[i];
if (before_decimal && c >= '0' && c <= '9')
{
++digit_num;
size_t offset = num_digits_before_decimal - digit_num;
if (offset && offset % 3 == 0)
{
result += "\033[4m";
result += c;
result += "\033[0m";
}
else
{
result += c;
}
}
else if (c == '.')
{
before_decimal = false;
result += c;
}
else
{
result += c;
}
}
return result;
}
void PrettyBlockOutputFormat::writeValueWithPadding(
const IColumn & column, const ISerialization & serialization, size_t row_num,
size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number)
@ -553,30 +489,6 @@ void PrettyBlockOutputFormat::writeSuffix()
}
}
void PrettyBlockOutputFormat::writeReadableNumberTip(const Chunk & chunk)
{
const auto & columns = chunk.getColumns();
auto is_single_number = readable_number_tip && chunk.getNumRows() == 1 && chunk.getNumColumns() == 1;
if (!is_single_number)
return;
if (columns[0]->isNullAt(0))
return;
auto value = columns[0]->getFloat64(0);
auto threshold = format_settings.pretty.output_format_pretty_single_large_number_tip_threshold;
if (threshold && isFinite(value) && abs(value) > threshold)
{
if (color)
writeCString("\033[90m", out);
writeCString(" -- ", out);
formatReadableQuantity(value, out, 2);
if (color)
writeCString("\033[0m", out);
}
}
void registerOutputFormatPretty(FormatFactory & factory)
{
registerPrettyFormatWithNoEscapesAndMonoBlock<PrettyBlockOutputFormat>(factory, "Pretty");

View File

@ -38,7 +38,6 @@ protected:
virtual void writeChunk(const Chunk & chunk, PortKind port_kind);
void writeMonoChunkIfNeeded();
void writeSuffix() override;
void writeReadableNumberTip(const Chunk & chunk);
void onRowsReadBeforeUpdate() override { total_rows = getRowsReadBefore(); }
@ -56,9 +55,9 @@ protected:
}
bool color;
bool readable_number_tip = false;
private:
bool readable_number_tip = false;
bool mono_block;
/// For mono_block == true only
Chunk mono_chunk;

View File

@ -4,6 +4,7 @@
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Formats/FormatFactory.h>
#include <Formats/PrettyFormatHelpers.h>
#include <Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h>
@ -189,7 +190,8 @@ void PrettyCompactBlockOutputFormat::writeRow(
}
writeCString(grid_symbols.bar, out);
writeReadableNumberTip(chunk);
if (readable_number_tip)
writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color);
writeCString("\n", out);
}

View File

@ -1,4 +1,5 @@
#include <Formats/FormatFactory.h>
#include <Formats/PrettyFormatHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h>
@ -102,7 +103,8 @@ 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);
if (readable_number_tip)
writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color);
writeChar('\n', out);
}

View File

@ -4,7 +4,10 @@
#include <IO/WriteBufferFromString.h>
#include <Processors/Formats/Impl/VerticalRowOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/PrettyFormatHelpers.h>
#include <Common/UTF8Helpers.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
@ -14,6 +17,8 @@ VerticalRowOutputFormat::VerticalRowOutputFormat(
WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_)
: IRowOutputFormat(header_, out_), format_settings(format_settings_)
{
color = format_settings.pretty.color == 1 || (format_settings.pretty.color == 2 && format_settings.is_writing_to_terminal);
const auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
@ -31,6 +36,7 @@ VerticalRowOutputFormat::VerticalRowOutputFormat(
}
names_and_paddings.resize(columns);
is_number.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
WriteBufferFromString buf(names_and_paddings[i]);
@ -42,6 +48,7 @@ VerticalRowOutputFormat::VerticalRowOutputFormat(
{
size_t new_size = max_name_width - name_widths[i] + names_and_paddings[i].size();
names_and_paddings[i].resize(new_size, ' ');
is_number[i] = isNumber(removeNullable(recursiveRemoveLowCardinality(sample.getByPosition(i).type)));
}
}
@ -61,7 +68,26 @@ void VerticalRowOutputFormat::writeField(const IColumn & column, const ISerializ
void VerticalRowOutputFormat::writeValue(const IColumn & column, const ISerialization & serialization, size_t row_num) const
{
serialization.serializeText(column, row_num, out, format_settings);
if (color && format_settings.pretty.highlight_digit_groups && is_number[field_number])
{
String serialized_value;
{
WriteBufferFromString buf(serialized_value);
serialization.serializeText(column, row_num, buf, format_settings);
}
/// Highlight groups of thousands.
serialized_value = highlightDigitGroups(serialized_value);
out.write(serialized_value.data(), serialized_value.size());
}
else
{
serialization.serializeText(column, row_num, out, format_settings);
}
/// Write a tip.
if (is_number[field_number])
writeReadableNumberTip(out, column, row_num, format_settings, color);
}

View File

@ -56,6 +56,9 @@ private:
using NamesAndPaddings = std::vector<String>;
NamesAndPaddings names_and_paddings;
std::vector<UInt8> is_number;
bool color;
};
}

View File

@ -597,12 +597,15 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
case (ActionsDAG::ActionType::COLUMN):
{
String name;
if (const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get()))
if (const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get());
column_const && column_const->getDataType() != TypeIndex::Function)
{
/// Re-generate column name for constant.
/// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8.
/// DAG from PK does not use it. This breaks matching by column name sometimes.
/// DAG from the query (with enabled analyzer) uses suffixes for constants, like 1_UInt8.
/// DAG from the PK does not use it. This breaks matching by column name sometimes.
/// Ideally, we should not compare names, but DAG subtrees instead.
name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName();
name = ASTLiteral(column_const->getField()).getColumnName();
}
else
name = node.result_name;

View File

@ -63,7 +63,7 @@ install_packages previous_release_package_folder
function save_settings_clean()
{
local out=$1 && shift
script -q -c "clickhouse-local -q \"select * from system.settings into outfile '$out'\"" --log-out /dev/null
script -q -c "clickhouse-local --implicit-select 0 -q \"select * from system.settings into outfile '$out'\"" --log-out /dev/null
}
# We save the (numeric) version of the old server to compare setting changes between the 2
@ -147,7 +147,6 @@ then
FROM new_settings
LEFT JOIN old_settings ON new_settings.name = old_settings.name
WHERE (new_value != old_value)
AND NOT (startsWith(new_value, 'auto(') AND old_value LIKE '%auto(%')
AND (name NOT IN (
SELECT arrayJoin(tupleElement(changes, 'name'))
FROM

View File

@ -18,7 +18,7 @@ FROM
reinterpretAsUInt8(
substring(
fuzzBits(
arrayStringConcat(arrayMap(x -> toString('\0'), range(10000))),
materialize(arrayStringConcat(arrayMap(x -> toString('\0'), range(10000)))),
0.3
),
id + 1,

View File

@ -1,5 +1,6 @@
SET session_timezone = 'UTC'; -- disable timezone randomization
SET enable_analyzer = 1; -- The old path formats the result with different whitespaces
SET output_format_pretty_single_large_number_tip_threshold = 0;
SELECT '-- Negative tests';
SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}

View File

@ -1,5 +1,6 @@
SET session_timezone = 'UTC'; -- disable timezone randomization
SET enable_analyzer = 1; -- The old path formats the result with different whitespaces
SET output_format_pretty_single_large_number_tip_threshold = 0;
SELECT '-- Negative tests';
SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}

View File

@ -1,3 +1,5 @@
SET output_format_pretty_single_large_number_tip_threshold = 0;
-- Appeared in https://github.com/ClickHouse/ClickHouse/pull/26978#issuecomment-890889362
WITH toDateTime('1970-06-17 07:39:21', 'Africa/Monrovia') as t
SELECT toUnixTimestamp(t),

View File

@ -4,5 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native. Expected string literal." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native." && echo 'OK' || echo 'FAIL' ||:

View File

@ -1,3 +1,5 @@
SET output_format_pretty_single_large_number_tip_threshold = 0;
drop table if exists src;
drop table if exists dst;
drop table if exists mv1;

View File

@ -1,5 +1,6 @@
-- Tags: no-fasttest
SET output_format_pretty_single_large_number_tip_threshold = 0;
SET enable_analyzer = 1;
SELECT

View File

@ -1,5 +1,7 @@
-- Tags: no-parallel, no-fasttest
SET output_format_pretty_single_large_number_tip_threshold = 0;
{# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #}
{% set ip4_generator = "select num::UInt32::IPv4 ip from (select arrayJoin(range(999999999, number)) as num from numbers(999999999,50)) order by ip" %}

View File

@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL "SELECT 101;"
$CLICKHOUSE_LOCAL "SELECT 102;SELECT 103;"
# Invalid SQL.
$CLICKHOUSE_LOCAL "SELECT 200; S" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_LOCAL --implicit-select 0 "SELECT 200; S" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_LOCAL "; SELECT 201;" 2>&1 | grep -o 'Empty query'
$CLICKHOUSE_LOCAL "; S; SELECT 202" 2>&1 | grep -o 'Empty query'

View File

@ -18,4 +18,4 @@ $CLICKHOUSE_LOCAL --query "SELECT 202;" --query "SELECT 202;"
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303"
$CLICKHOUSE_LOCAL --query "" --query ""
$CLICKHOUSE_LOCAL --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_LOCAL --implicit-select 0 --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error'

View File

@ -1,2 +1,3 @@
allow_introspection_functions 1
storage_file_read_method mmap
implicit_select 1

View File

@ -1,14 +1,14 @@
<1: created view> a
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x
<2: refreshed> 3 1 1
<3: time difference at least> 1000
<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 1 3 3 3 0
<4.5: altered> Scheduled 2050-01-01 00:00:01 2052-01-01 00:00:00
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 2 AS x\nFROM default.src
<5: no refresh> 3
<6: refreshed> 2
<7: refreshed> Scheduled 2052-02-03 04:05:06 2054-01-01 00:00:00
CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a
CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a
<7.5: created dependent> 2052-11-11 11:11:11
<8: refreshed> 20
<9: refreshed> a Scheduled 2054-01-01 00:00:00
@ -26,4 +26,4 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(
<17: chain-refreshed> a Scheduled 2062-01-01 00:00:00
<17: chain-refreshed> b Scheduled 2062-01-01 00:00:00
<18: removed dependency> b Scheduled 2062-03-03 03:03:03 2062-03-03 03:03:03 2064-01-01 00:00:00
CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a
CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a

View File

@ -7,9 +7,9 @@
<25: rename during refresh> f Running
<27: cancelled> f Scheduled cancelled
<28: drop during refresh> 0 0
CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x
CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x
<29: randomize> 1 1
CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src
CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS x\nFROM default.src
<30: to existing table> 10
<31: to existing table> 10
<31: to existing table> 20

View File

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

View File

@ -0,0 +1,13 @@
SET enable_analyzer = 1;
SELECT arrayMap(x -> x, [1, 2, 3]) AS x, isConstant(x);
SELECT arrayMap(x -> x + 1, [1, 2, 3]) AS x, isConstant(x);
SELECT arrayMap(x -> x + x, [1, 2, 3]) AS x, isConstant(x);
SELECT arrayMap((x, y) -> x + y, [1, 2, 3], [4, 5, 6]) AS x, isConstant(x);
SELECT arrayMap(x -> 1, [1, 2, 3]) AS x, isConstant(x);
SELECT arrayMap(x -> x + number, [1, 2, 3]) AS x, isConstant(x) FROM numbers(1);
SELECT arrayMap(x -> number, [1, 2, 3]) AS x, isConstant(x) FROM numbers(1);
SELECT arrayMax([1, 2, 3]) AS x, isConstant(x);
-- Does not work yet:
-- SELECT [1, 2, 3] IN arrayMap(x -> x, [1, 2, 3]);

View File

@ -0,0 +1,5 @@
3
3
3
Syntax error
3

View File

@ -0,0 +1,11 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_LOCAL "1 + 2"
$CLICKHOUSE_LOCAL -q "1 + 2"
$CLICKHOUSE_LOCAL --query "1 + 2"
$CLICKHOUSE_LOCAL --implicit_select 0 --query "1 + 2" 2>&1 | grep -oF 'Syntax error'
$CLICKHOUSE_LOCAL --implicit_select 0 --query "SELECT 1 + 2"

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,11 @@
SET output_format_pretty_color = 1, output_format_pretty_highlight_digit_groups = 1, output_format_pretty_single_large_number_tip_threshold = 1;
SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical;
SET output_format_pretty_color = 0, output_format_pretty_highlight_digit_groups = 1, output_format_pretty_single_large_number_tip_threshold = 1;
SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical;
SET output_format_pretty_color = 1, output_format_pretty_highlight_digit_groups = 0, output_format_pretty_single_large_number_tip_threshold = 1;
SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical;
SET output_format_pretty_color = 0, output_format_pretty_highlight_digit_groups = 0, output_format_pretty_single_large_number_tip_threshold = 0;
SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical;

View File

@ -186,7 +186,6 @@ ComplexKeyCache
ComplexKeyDirect
ComplexKeyHashed
Composable
composable
ConcurrencyControlAcquired
ConcurrencyControlSoftLimit
Config
@ -405,12 +404,12 @@ ITION
Identifiant
IdentifierQuotingRule
IdentifierQuotingStyle
Incrementing
IndexesAreNeighbors
InfluxDB
InJodaSyntax
InJodaSyntaxOrNull
InJodaSyntaxOrZero
Incrementing
IndexesAreNeighbors
InfluxDB
Instana
IntN
Integrations
@ -1475,6 +1474,7 @@ combinator
combinators
comparising
composable
composable
compressability
concat
concatAssumeInjective
@ -2355,6 +2355,7 @@ parsedatetime
parsers
partitionID
partitionId
partsupp
pathFull
pclmulqdq
pcre