Merge branch 'ClickHouse:master' into variant_inference

This commit is contained in:
Shaun Struwig 2024-05-15 19:27:49 +02:00 committed by GitHub
commit dacbe1a427
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 151 additions and 42 deletions

View File

@ -505,7 +505,7 @@ ASTPtr InterpreterCreateQuery::formatProjections(const ProjectionsDescription &
}
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
const ASTExpressionList & columns_ast, ContextPtr context_, LoadingStrictnessLevel mode)
const ASTExpressionList & columns_ast, ContextPtr context_, LoadingStrictnessLevel mode, bool is_restore_from_backup)
{
/// First, deduce implicit types.
@ -514,7 +514,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
NamesAndTypesList column_names_and_types;
bool make_columns_nullable = mode <= LoadingStrictnessLevel::CREATE && context_->getSettingsRef().data_type_default_nullable;
bool make_columns_nullable = mode <= LoadingStrictnessLevel::SECONDARY_CREATE && !is_restore_from_backup && context_->getSettingsRef().data_type_default_nullable;
bool has_columns_with_default_without_type = false;
for (const auto & ast : columns_ast.children)
@ -694,7 +694,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
res.add(std::move(column));
}
if (mode <= LoadingStrictnessLevel::CREATE && context_->getSettingsRef().flatten_nested)
if (mode <= LoadingStrictnessLevel::SECONDARY_CREATE && !is_restore_from_backup && context_->getSettingsRef().flatten_nested)
res.flattenNested();
@ -739,7 +739,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
if (create.columns_list->columns)
{
properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), mode);
properties.columns = getColumnsDescription(*create.columns_list->columns, getContext(), mode, is_restore_from_backup);
}
if (create.columns_list->indices)

View File

@ -74,7 +74,7 @@ public:
/// Obtain information about columns, their types, default values and column comments,
/// for case when columns in CREATE query is specified explicitly.
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, LoadingStrictnessLevel mode);
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, LoadingStrictnessLevel mode, bool is_restore_from_backup = false);
static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints);
static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name);

View File

@ -107,6 +107,9 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!columns_p.parse(pos, columns, expected))
return false;
/// Optional trailing comma
ParserToken(TokenType::Comma).ignore(pos);
if (!s_rparen.ignore(pos, expected))
return false;
}

View File

@ -4,20 +4,6 @@
namespace DB
{
Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skip_insignificant)
{
Lexer lexer(begin, end, max_query_size);
bool stop = false;
do
{
Token token = lexer.nextToken();
stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded;
if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant()))
data.emplace_back(std::move(token));
} while (!stop);
}
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin)
{
/// We have just two kind of parentheses: () and [].

View File

@ -15,25 +15,44 @@ namespace DB
*/
/** Used as an input for parsers.
* All whitespace and comment tokens are transparently skipped.
* All whitespace and comment tokens are transparently skipped if `skip_insignificant`.
*/
class Tokens
{
private:
std::vector<Token> data;
std::size_t last_accessed_index = 0;
Lexer lexer;
bool skip_insignificant;
public:
Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skip_insignificant = true);
ALWAYS_INLINE inline const Token & operator[](size_t index)
Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skip_insignificant_ = true)
: lexer(begin, end, max_query_size), skip_insignificant(skip_insignificant_)
{
assert(index < data.size());
last_accessed_index = std::max(last_accessed_index, index);
return data[index];
}
ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; }
const Token & operator[] (size_t index)
{
while (true)
{
if (index < data.size())
return data[index];
if (!data.empty() && data.back().isEnd())
return data.back();
Token token = lexer.nextToken();
if (!skip_insignificant || token.isSignificant())
data.emplace_back(token);
}
}
const Token & max()
{
if (data.empty())
return (*this)[0];
return data.back();
}
};

View File

@ -13,10 +13,14 @@ namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
RegexpFieldExtractor::RegexpFieldExtractor(const FormatSettings & format_settings) : regexp(format_settings.regexp.regexp), skip_unmatched(format_settings.regexp.skip_unmatched)
RegexpFieldExtractor::RegexpFieldExtractor(const FormatSettings & format_settings) : regexp_str(format_settings.regexp.regexp), regexp(regexp_str), skip_unmatched(format_settings.regexp.skip_unmatched)
{
if (regexp_str.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The regular expression is not set for the `Regexp` format. It requires setting the value of the `format_regexp` setting.");
size_t fields_count = regexp.NumberOfCapturingGroups();
matched_fields.resize(fields_count);
re2_arguments.resize(fields_count);
@ -58,8 +62,8 @@ bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf)
static_cast<int>(re2_arguments_ptrs.size()));
if (!match && !skip_unmatched)
throw Exception(ErrorCodes::INCORRECT_DATA, "Line \"{}\" doesn't match the regexp.",
std::string(buf.position(), line_to_match));
throw Exception(ErrorCodes::INCORRECT_DATA, "Line \"{}\" doesn't match the regexp: `{}`",
std::string(buf.position(), line_to_match), regexp_str);
buf.position() += line_size;
if (!buf.eof() && !checkChar('\n', buf))

View File

@ -31,6 +31,7 @@ public:
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
private:
String regexp_str;
const re2::RE2 regexp;
// The vector of fields extracted from line using regexp.
std::vector<std::string_view> matched_fields;

View File

@ -572,9 +572,16 @@ bool ValuesBlockInputFormat::checkDelimiterAfterValue(size_t column_idx)
skipWhitespaceIfAny(*buf);
if (likely(column_idx + 1 != num_columns))
{
return checkChar(',', *buf);
}
else
{
/// Optional trailing comma.
if (checkChar(',', *buf))
skipWhitespaceIfAny(*buf);
return checkChar(')', *buf);
}
}
bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx)

View File

@ -1806,7 +1806,7 @@ def _upload_build_profile_data(
address,
size,
type,
symbol,
symbol
)
SELECT {pr_info.number}, '{pr_info.sha}', '{job_report.start_time}', '{build_name}', '{instance_type}', '{instance_id}',
file, reinterpretAsUInt64(reverse(unhex(address))), reinterpretAsUInt64(reverse(unhex(size))), type, symbol

View File

@ -1,13 +1,14 @@
import pytest
from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION
from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION, is_arm
# For arm version see https://github.com/ClickHouse/ClickHouse/pull/59132
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
with_zookeeper=False,
image="clickhouse/clickhouse-server",
tag=CLICKHOUSE_CI_MIN_TESTED_VERSION,
tag="24.1" if is_arm() else CLICKHOUSE_CI_MIN_TESTED_VERSION,
stay_alive=True,
with_installed_binary=True,
)
@ -15,7 +16,7 @@ node2 = cluster.add_instance(
"node2",
with_zookeeper=False,
image="clickhouse/clickhouse-server",
tag=CLICKHOUSE_CI_MIN_TESTED_VERSION,
tag="24.1" if is_arm() else CLICKHOUSE_CI_MIN_TESTED_VERSION,
stay_alive=True,
with_installed_binary=True,
)

View File

@ -23,13 +23,23 @@ def ch_cluster():
try:
cluster.start()
os.system(
"docker cp {local} {cont_id}:{dist}".format(
local=os.path.join(SCRIPT_DIR, "model/."),
cont_id=instance.docker_id,
dist="/etc/clickhouse-server/model",
instance.exec_in_container(["mkdir", f"/etc/clickhouse-server/model/"])
machine = instance.get_machine_name()
for source_name in os.listdir(os.path.join(SCRIPT_DIR, "model/.")):
dest_name = source_name
if machine in source_name:
machine_suffix = "_" + machine
dest_name = source_name[: -len(machine_suffix)]
os.system(
"docker cp {local} {cont_id}:{dist}".format(
local=os.path.join(SCRIPT_DIR, f"model/{source_name}"),
cont_id=instance.docker_id,
dist=f"/etc/clickhouse-server/model/{dest_name}",
)
)
)
instance.restart_clickhouse()
yield cluster

View File

@ -0,0 +1,17 @@
<clickhouse>
<storage_configuration>
<disks>
<disk_s3>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</disk_s3>
<disk_encrypted>
<type>encrypted</type>
<disk>disk_s3</disk>
<key>1234567812345678</key>
</disk_encrypted>
</disks>
</storage_configuration>
</clickhouse>

View File

@ -19,7 +19,9 @@ def cluster():
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node",
main_configs=["configs/storage.xml"],
main_configs=["configs/storage_arm.xml"]
if is_arm()
else ["configs/storage_amd.xml"],
with_minio=True,
with_hdfs=not is_arm(),
)

View File

@ -79,6 +79,29 @@ def started_cluster():
cluster.shutdown()
def test_flatten_nested(started_cluster):
main_node.query(
"CREATE DATABASE create_replicated_table ENGINE = Replicated('/test/create_replicated_table', 'shard1', 'replica' || '1');"
)
dummy_node.query(
"CREATE DATABASE create_replicated_table ENGINE = Replicated('/test/create_replicated_table', 'shard1', 'replica2');"
)
main_node.query(
"CREATE TABLE create_replicated_table.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);"
)
main_node.query(
"CREATE MATERIALIZED VIEW create_replicated_table.mv ENGINE=ReplicatedMergeTree ORDER BY tuple() AS select d, cast([(k, toString(i32))] as Nested(a UInt64, b String)) from create_replicated_table.replicated_table"
)
assert main_node.query(
"show create create_replicated_table.mv"
) == dummy_node.query("show create create_replicated_table.mv")
main_node.query("DROP DATABASE create_replicated_table SYNC")
dummy_node.query("DROP DATABASE create_replicated_table SYNC")
def test_create_replicated_table(started_cluster):
main_node.query(
"CREATE DATABASE create_replicated_table ENGINE = Replicated('/test/create_replicated_table', 'shard1', 'replica' || '1');"

View File

@ -0,0 +1,4 @@
CREATE TEMPORARY TABLE test (a UInt8, b UInt8, c UInt8);
INSERT INTO test (a, b, c, ) VALUES (1, 2, 3);
INSERT INTO test (a, b, c) VALUES (4, 5, 6);
SELECT * FROM test ORDER BY a;

View File

@ -0,0 +1,2 @@
regular expression is not set
`Upyachka`

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-ordinary-database, long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_LOCAL} --query "SELECT * FROM format(Regexp, 's String', 'Hello')" 2>&1 | grep -o -F 'regular expression is not set'
${CLICKHOUSE_LOCAL} --query "SELECT * FROM format(Regexp, 's String', 'Hello') SETTINGS format_regexp = 'Upyachka'" 2>&1 | grep -o -F '`Upyachka`'

View File

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

View File

@ -0,0 +1,5 @@
CREATE TEMPORARY TABLE test (a UInt8, b UInt8, c UInt8);
INSERT INTO test (a, b, c) VALUES (1, 2, 3, );
INSERT INTO test (a, b, c) VALUES (4, 5, 6,);
INSERT INTO test (a, b, c) VALUES (7, 8, 9);
SELECT * FROM test ORDER BY a;

View File

@ -0,0 +1,11 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-ordinary-database, long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# In previous versions this command took longer than ten minutes. Now it takes less than a second in release mode:
python3 -c 'import sys; import struct; sys.stdout.buffer.write(b"".join(struct.pack("<Q", 36) + b"\x40" + f"{i:064}".encode("ascii") for i in range(1024 * 1024)))' |
${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&max_query_size=100000000&query=INSERT+INTO+FUNCTION+null('timestamp+UInt64,+label+String')+FORMAT+RowBinary" --data-binary @-