Merge pull request #52503 from ClickHouse/more-strict-tryparse

Stricter `tryParse`
This commit is contained in:
Alexey Milovidov 2024-08-15 21:58:22 +00:00 committed by GitHub
commit 3508667e6c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 126 additions and 52 deletions

View File

@ -129,6 +129,7 @@ configure
# Check that all new/changed setting were added in settings changes history.
# Some settings can be different for builds with sanitizers, so we check
# Also the automatic value of 'max_threads' and similar was displayed as "'auto(...)'" in previous versions instead of "auto(...)".
# settings changes only for non-sanitizer builds.
IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'")
if [ "${IS_SANITIZED}" -eq "0" ]
@ -145,7 +146,9 @@ then
old_settings.value AS old_value
FROM new_settings
LEFT JOIN old_settings ON new_settings.name = old_settings.name
WHERE (new_settings.value != old_settings.value) AND (name NOT IN (
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
(
@ -177,7 +180,7 @@ then
if [ -s changed_settings.txt ]
then
mv changed_settings.txt /test_output/
echo -e "Changed settings are not reflected in settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv
echo -e "Changed settings are not reflected in the settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv
else
echo -e "There are no changed settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv
fi

View File

@ -93,7 +93,7 @@ namespace
break;
}
UUID id = parse<UUID>(line);
UUID id = parse<UUID>(line.substr(0, line.find('\t')));
line.clear();
String queries;

View File

@ -44,7 +44,7 @@ namespace ErrorCodes
namespace zkutil
{
/// Preferred size of multi() command (in number of ops)
/// Preferred size of multi command (in the number of operations)
constexpr size_t MULTI_BATCH_SIZE = 100;
struct ShuffleHost

View File

@ -79,11 +79,16 @@ std::vector<String> parseRemoteDescription(
/// Look for the corresponding closing bracket
for (m = i + 1; m < r; ++m)
{
if (description[m] == '{') ++cnt;
if (description[m] == '}') --cnt;
if (description[m] == '.' && description[m-1] == '.') last_dot = m;
if (description[m] == separator) have_splitter = true;
if (cnt == 0) break;
if (description[m] == '{')
++cnt;
if (description[m] == '}')
--cnt;
if (description[m] == '.' && description[m-1] == '.')
last_dot = m;
if (description[m] == separator)
have_splitter = true;
if (cnt == 0)
break;
}
if (cnt != 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': incorrect brace sequence in first argument", func_name);

View File

@ -54,7 +54,7 @@ namespace
std::filesystem::path path(snapshot_path);
std::string filename = path.stem();
Strings name_parts;
splitInto<'_'>(name_parts, filename);
splitInto<'_', '.'>(name_parts, filename);
return parse<uint64_t>(name_parts[1]);
}

View File

@ -26,12 +26,16 @@ std::optional<RaftServerConfig> RaftServerConfig::parse(std::string_view server)
if (!with_id_endpoint && !with_server_type && !with_priority)
return std::nullopt;
const std::string_view id_str = parts[0];
std::string_view id_str = parts[0];
if (!id_str.starts_with("server."))
return std::nullopt;
id_str = id_str.substr(7);
if (auto eq_pos = id_str.find('='); std::string_view::npos != eq_pos)
id_str = id_str.substr(0, eq_pos);
Int32 id;
if (!tryParse(id, std::next(id_str.begin(), 7)))
if (!tryParse(id, id_str))
return std::nullopt;
if (id <= 0)
return std::nullopt;

View File

@ -24,9 +24,7 @@ void GTIDSet::tryMerge(size_t i)
void GTIDSets::parse(String gtid_format)
{
if (gtid_format.empty())
{
return;
}
std::vector<String> gtid_sets;
boost::split(gtid_sets, gtid_format, [](char c) { return c == ','; });

View File

@ -10,20 +10,19 @@ GTEST_TEST(GTIDSetsContains, Tests)
contained1, contained2, contained3, contained4, contained5,
not_contained1, not_contained2, not_contained3, not_contained4, not_contained5, not_contained6;
gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60");
contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60");
gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60");
contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60");
contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:2-3:11:47-49");
contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:11");
contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:47-49:60");
contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:60");
contained4.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:2-16:47-49:60");
contained5.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:60");
not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60");
not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60");
not_contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:0-3:11:47-49");
not_contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:99");
not_contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:46-49:60");
not_contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:99");
not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60, 00000000-0000-0000-0000-000000000000");
not_contained4.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:2-16:46-49:60");
not_contained5.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:99");
not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60, 00000000-0000-0000-0000-000000000000");
ASSERT_TRUE(gtid_set.contains(contained1));
ASSERT_TRUE(gtid_set.contains(contained2));

View File

@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f)
String SettingFieldMaxThreads::toString() const
{
if (is_auto)
return "'auto(" + ::DB::toString(value) + ")'";
return "auto(" + ::DB::toString(value) + ")";
else
return ::DB::toString(value);
}

View File

@ -153,7 +153,7 @@ struct SettingFieldMaxThreads
operator UInt64() const { return value; } /// NOLINT
explicit operator Field() const { return value; }
/// Writes "auto(<number>)" instead of simple "<number>" if `is_auto==true`.
/// Writes "auto(<number>)" instead of simple "<number>" if `is_auto == true`.
String toString() const;
void parseFromString(const String & str);

View File

@ -1,4 +1,5 @@
#pragma once
#include <map>
#include <mutex>
#include <unordered_map>

View File

@ -258,6 +258,20 @@ inline void readBoolText(bool & x, ReadBuffer & buf)
char tmp = '0';
readChar(tmp, buf);
x = tmp != '0';
if (!buf.eof() && isAlphaASCII(tmp))
{
if (tmp == 't' || tmp == 'T')
{
assertStringCaseInsensitive("rue", buf);
x = true;
}
else if (tmp == 'f' || tmp == 'F')
{
assertStringCaseInsensitive("alse", buf);
x = false;
}
}
}
template <typename ReturnType = void>
@ -1735,6 +1749,7 @@ inline T parse(const char * data, size_t size)
T res;
ReadBufferFromMemory buf(data, size);
readText(res, buf);
assertEOF(buf);
return res;
}
@ -1742,7 +1757,9 @@ template <typename T>
inline bool tryParse(T & res, const char * data, size_t size)
{
ReadBufferFromMemory buf(data, size);
return tryReadText(res, buf);
if (!tryReadText(res, buf))
return false;
return buf.eof();
}
template <typename T>

View File

@ -230,21 +230,37 @@ String Cluster::Address::toFullString(bool use_compact_format) const
}
}
Cluster::Address Cluster::Address::fromFullString(const String & full_string)
Cluster::Address Cluster::Address::fromFullString(std::string_view full_string)
{
const char * address_begin = full_string.data();
const char * address_end = address_begin + full_string.size();
const char * user_pw_end = strchr(full_string.data(), '@');
std::string_view user_password;
if (auto pos = full_string.find('@'); pos != std::string_view::npos)
user_password = full_string.substr(pos + 1);
/// parsing with the new shard{shard_index}[_replica{replica_index}] format
if (!user_pw_end && startsWith(full_string, "shard"))
if (user_password.empty() && full_string.starts_with("shard"))
{
const char * underscore = strchr(full_string.data(), '_');
Address address;
address.shard_index = parse<UInt32>(address_begin + strlen("shard"));
address.replica_index = underscore ? parse<UInt32>(underscore + strlen("_replica")) : 0;
if (auto underscore_pos = full_string.find('_'); underscore_pos != std::string_view::npos)
{
address.shard_index = parse<UInt32>(full_string.substr(0, underscore_pos).substr(strlen("shard")));
if (full_string.substr(underscore_pos + 1).starts_with("replica"))
{
address.replica_index = parse<UInt32>(full_string.substr(underscore_pos + 1 + strlen("replica")));
}
else if (full_string.substr(underscore_pos + 1).starts_with("all_replicas"))
{
address.replica_index = 0;
}
else
throw Exception(ErrorCodes::SYNTAX_ERROR, "Incorrect address '{}', should be in a form of `shardN_all_replicas` or `shardN_replicaM`", full_string);
}
else
{
address.shard_index = parse<UInt32>(full_string.substr(strlen("shard")));
address.replica_index = 0;
}
return address;
}
@ -255,9 +271,13 @@ Cluster::Address Cluster::Address::fromFullString(const String & full_string)
/// - credentials are exposed in file name;
/// - the file name can be too long.
const char * address_begin = full_string.data();
const char * address_end = address_begin + full_string.size();
const char * user_pw_end = strchr(address_begin, '@');
Protocol::Secure secure = Protocol::Secure::Disable;
const char * secure_tag = "+secure";
if (endsWith(full_string, secure_tag))
if (full_string.ends_with(secure_tag))
{
address_end -= strlen(secure_tag);
secure = Protocol::Secure::Enable;

View File

@ -168,7 +168,7 @@ public:
String toFullString(bool use_compact_format) const;
/// Returns address with only shard index and replica index or full address without shard index and replica index
static Address fromFullString(const String & address_full_string);
static Address fromFullString(std::string_view full_string);
/// Returns resolved address if it does resolve.
std::optional<Poco::Net::SocketAddress> getResolvedAddress() const;

View File

@ -332,6 +332,8 @@ struct DeltaLakeMetadataImpl
WhichDataType which(check_type->getTypeId());
if (which.isStringOrFixedString())
return value;
else if (isBool(check_type))
return parse<bool>(value);
else if (which.isInt8())
return parse<Int8>(value);
else if (which.isUInt8())

View File

@ -1,4 +1,4 @@
#include "StorageExternalDistributed.h"
#include <Storages/StorageExternalDistributed.h>
#include <Core/Settings.h>
#include <Storages/StorageFactory.h>
@ -6,6 +6,8 @@
#include <Interpreters/InterpreterSelectQuery.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/parseAddress.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Common/parseRemoteDescription.h>
@ -112,14 +114,39 @@ void registerStorageExternalDistributed(StorageFactory & factory)
std::unordered_set<StoragePtr> shards;
ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end());
ASTPtr * address_arg = nullptr;
/// If there is a named collection argument, named `addresses_expr`
for (auto & node : inner_engine_args)
{
if (ASTFunction * func = node->as<ASTFunction>(); func && func->name == "equals" && func->arguments)
{
if (ASTExpressionList * func_args = func->arguments->as<ASTExpressionList>(); func_args && func_args->children.size() == 2)
{
if (ASTIdentifier * arg_name = func_args->children[0]->as<ASTIdentifier>(); arg_name && arg_name->name() == "addresses_expr")
{
address_arg = &func_args->children[1];
break;
}
}
}
}
/// Otherwise it is the first argument.
if (!address_arg)
address_arg = &inner_engine_args.at(0);
String addresses_expr = checkAndGetLiteralArgument<String>(*address_arg, "addresses");
Strings shards_addresses = get_addresses(addresses_expr);
auto engine_name = checkAndGetLiteralArgument<String>(engine_args[0], "engine_name");
if (engine_name == "URL")
{
auto configuration = StorageURL::getConfiguration(inner_engine_args, context);
auto shards_addresses = get_addresses(configuration.addresses_expr);
auto format_settings = StorageURL::getFormatSettingsFromArgs(args);
for (const auto & shard_address : shards_addresses)
{
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StorageURL::getConfiguration(inner_engine_args, context);
auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses);
if (uri_options.size() > 1)
{
@ -140,13 +167,12 @@ void registerStorageExternalDistributed(StorageFactory & factory)
else if (engine_name == "MySQL")
{
MySQLSettings mysql_settings;
auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings);
auto shards_addresses = get_addresses(configuration.addresses_expr);
for (const auto & shard_address : shards_addresses)
{
auto current_configuration{configuration};
current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306);
auto pool = createMySQLPoolWithFailover(current_configuration, mysql_settings);
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings);
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306);
auto pool = createMySQLPoolWithFailover(configuration, mysql_settings);
shards.insert(std::make_shared<StorageMySQL>(
args.table_id, std::move(pool), configuration.database, configuration.table,
/* replace_query = */ false, /* on_duplicate_clause = */ "",
@ -157,14 +183,13 @@ void registerStorageExternalDistributed(StorageFactory & factory)
#if USE_LIBPQXX
else if (engine_name == "PostgreSQL")
{
auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context);
auto shards_addresses = get_addresses(configuration.addresses_expr);
for (const auto & shard_address : shards_addresses)
{
auto current_configuration{configuration};
current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432);
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context);
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432);
auto pool = std::make_shared<postgres::PoolWithFailover>(
current_configuration,
configuration,
settings.postgresql_connection_pool_size,
settings.postgresql_connection_pool_wait_timeout,
settings.postgresql_connection_pool_retries,

View File

@ -46,7 +46,7 @@ def test_cgroup_cpu_limit():
"clickhouse local -q \"select value from system.settings where name='max_threads'\"",
num_cpus,
)
expect_output = (r"\'auto({})\'".format(math.ceil(num_cpus))).encode()
expect_output = (r"auto({})".format(math.ceil(num_cpus))).encode()
assert (
result.strip() == expect_output
), f"fail for cpu limit={num_cpus}, result={result.strip()}, expect={expect_output}"