Merge branch 'master' into revert-68023-revert-65860-atomic-database-in-clickhouse-local

This commit is contained in:
Alexey Milovidov 2024-11-11 01:19:54 +01:00
commit cef2e4a1c0
49 changed files with 1914 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

@ -24,7 +24,7 @@ All expressions in a query that have the same AST (the same record or same resul
## Types of Results
All functions return a single return as the result (not several values, and not zero values). The type of result is usually defined only by the types of arguments, not by the values. Exceptions are the tupleElement function (the a.N operator), and the toFixedString function.
All functions return a single value as the result (not several values, and not zero values). The type of result is usually defined only by the types of arguments, not by the values. Exceptions are the tupleElement function (the a.N operator), and the toFixedString function.
## Constants

View File

@ -23,7 +23,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>
@ -32,7 +31,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>
@ -71,9 +69,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 +126,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,8 +151,27 @@ public:
auto function = std::make_unique<FunctionExpression>(expression_actions, types, names,
capture->return_type, capture->return_name);
/// 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:
ExpressionActionsPtr expression_actions;
@ -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
{
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