Merge branch 'master' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-09-11 14:03:35 +02:00 committed by GitHub
commit c53e165118
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 69 additions and 23 deletions

View File

@ -187,10 +187,15 @@ function setup_logs_replication
') ')
echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2 echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2
echo "::group::${table}" echo "::group::${table}"
# there's the only way big "$statement" can be printed without causing EAGAIN error # there's the only way big "$statement" can be printed without causing EAGAIN error
# cat: write error: Resource temporarily unavailable # cat: write error: Resource temporarily unavailable
echo "$statement" | cat statement_print="${statement}"
if [ "${#statement_print}" -gt 4000 ]; then
statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}"
fi
echo -e "$statement_print"
echo "::endgroup::" echo "::endgroup::"
echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \

View File

@ -116,15 +116,17 @@ class GroupConcatImpl final
SerializationPtr serialization; SerializationPtr serialization;
UInt64 limit; UInt64 limit;
const String delimiter; const String delimiter;
const DataTypePtr type;
public: public:
GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_) GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_)
: IAggregateFunctionDataHelper<GroupConcatData<has_limit>, GroupConcatImpl<has_limit>>( : IAggregateFunctionDataHelper<GroupConcatData<has_limit>, GroupConcatImpl<has_limit>>(
{data_type_}, parameters_, std::make_shared<DataTypeString>()) {data_type_}, parameters_, std::make_shared<DataTypeString>())
, serialization(this->argument_types[0]->getDefaultSerialization())
, limit(limit_) , limit(limit_)
, delimiter(delimiter_) , delimiter(delimiter_)
, type(data_type_)
{ {
serialization = isFixedString(type) ? std::make_shared<DataTypeString>()->getDefaultSerialization() : this->argument_types[0]->getDefaultSerialization();
} }
String getName() const override { return name; } String getName() const override { return name; }
@ -140,7 +142,14 @@ public:
if (cur_data.data_size != 0) if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.insert(columns[0], serialization, row_num, arena); if (isFixedString(type))
{
ColumnWithTypeAndName col = {columns[0]->getPtr(), type, "column"};
const auto & col_str = castColumn(col, std::make_shared<DataTypeString>());
cur_data.insert(col_str.get(), serialization, row_num, arena);
}
else
cur_data.insert(columns[0], serialization, row_num, arena);
} }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override

View File

@ -148,6 +148,7 @@ namespace DB
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \ M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \ M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
M(Bool, database_replicated_allow_detach_permanently, true, "Allow detaching tables permanently in Replicated databases", 0) \
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \ M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \ M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \ M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \

View File

@ -63,6 +63,7 @@ namespace ErrorCodes
extern const int NO_ACTIVE_REPLICAS; extern const int NO_ACTIVE_REPLICAS;
extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT; extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT;
extern const int CANNOT_RESTORE_TABLE; extern const int CANNOT_RESTORE_TABLE;
extern const int SUPPORT_IS_DISABLED;
} }
static constexpr const char * REPLICATED_DATABASE_MARK = "DatabaseReplicated"; static constexpr const char * REPLICATED_DATABASE_MARK = "DatabaseReplicated";
@ -1741,6 +1742,9 @@ void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const
{ {
waitDatabaseStarted(); waitDatabaseStarted();
if (!local_context->getServerSettings().database_replicated_allow_detach_permanently)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for DETACH TABLE PERMANENTLY is disabled");
auto txn = local_context->getZooKeeperMetadataTransaction(); auto txn = local_context->getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn); assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery()) if (txn && txn->isInitialQuery())

View File

@ -124,7 +124,7 @@ public:
std::string_view sqid = col_non_const->getDataAt(i).toView(); std::string_view sqid = col_non_const->getDataAt(i).toView();
std::vector<UInt64> integers = sqids.decode(String(sqid)); std::vector<UInt64> integers = sqids.decode(String(sqid));
res_nested_data.insert(integers.begin(), integers.end()); res_nested_data.insert(integers.begin(), integers.end());
res_offsets_data.push_back(integers.size()); res_offsets_data.push_back(res_offsets_data.back() + integers.size());
} }
} }
else else

View File

@ -36,7 +36,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax)
/// Case when bucket name represented in domain name of S3 URL. /// Case when bucket name represented in domain name of S3 URL.
/// E.g. (https://bucket-name.s3.region.amazonaws.com/key) /// E.g. (https://bucket-name.s3.region.amazonaws.com/key)
/// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access
static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); static const RE2 virtual_hosted_style_pattern(R"(([^.]+)\.(s3express[\-a-z0-9]+|s3|cos|obs|.*oss[^\/]*|eos)([.\-][a-z0-9\-.:]+))");
/// Case when AWS Private Link Interface is being used /// Case when AWS Private Link Interface is being used
/// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key)

View File

@ -204,6 +204,14 @@ TEST(S3UriTest, validPatterns)
ASSERT_EQ("", uri.version_id); ASSERT_EQ("", uri.version_id);
ASSERT_EQ(true, uri.is_virtual_hosted_style); ASSERT_EQ(true, uri.is_virtual_hosted_style);
} }
{
S3::URI uri("https://bucket-test.cn-beijing-internal.oss-data-acc.aliyuncs.com/cc-2zeh496zqm0g6e09g");
ASSERT_EQ("https://cn-beijing-internal.oss-data-acc.aliyuncs.com", uri.endpoint);
ASSERT_EQ("bucket-test", uri.bucket);
ASSERT_EQ("cc-2zeh496zqm0g6e09g", uri.key);
ASSERT_EQ("", uri.version_id);
ASSERT_EQ(true, uri.is_virtual_hosted_style);
}
} }
TEST(S3UriTest, versionIdChecks) TEST(S3UriTest, versionIdChecks)

View File

@ -258,7 +258,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
query_context, query_context,
query_, query_,
client_info, client_info,
priorities.insert(static_cast<int>(settings.priority)), priorities.insert(settings.priority),
std::move(thread_group), std::move(thread_group),
query_kind, query_kind,
settings, settings,

View File

@ -31,7 +31,7 @@ namespace DB
class QueryPriorities class QueryPriorities
{ {
public: public:
using Priority = int; using Priority = size_t;
private: private:
friend struct Handle; friend struct Handle;

View File

@ -64,9 +64,7 @@ namespace
void checkAndAdjustSettings( void checkAndAdjustSettings(
ObjectStorageQueueSettings & queue_settings, ObjectStorageQueueSettings & queue_settings,
ASTStorage * engine_args, bool is_attach)
bool is_attach,
const LoggerPtr & log)
{ {
if (!is_attach && !queue_settings.mode.changed) if (!is_attach && !queue_settings.mode.changed)
{ {
@ -85,16 +83,6 @@ namespace
"Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})", "Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})",
queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms); queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms);
} }
if (!is_attach && !queue_settings.processing_threads_num.changed)
{
queue_settings.processing_threads_num = std::max<uint32_t>(getNumberOfPhysicalCPUCores(), 16);
engine_args->settings->as<ASTSetQuery>()->changes.insertSetting(
"processing_threads_num",
queue_settings.processing_threads_num.value);
LOG_TRACE(log, "Set `processing_threads_num` to {}", queue_settings.processing_threads_num);
}
} }
std::shared_ptr<ObjectStorageQueueLog> getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings) std::shared_ptr<ObjectStorageQueueLog> getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings)
@ -130,7 +118,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
const String & comment, const String & comment,
ContextPtr context_, ContextPtr context_,
std::optional<FormatSettings> format_settings_, std::optional<FormatSettings> format_settings_,
ASTStorage * engine_args, ASTStorage * /* engine_args */,
LoadingStrictnessLevel mode) LoadingStrictnessLevel mode)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_) , WithContext(context_)
@ -154,7 +142,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs"); throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs");
} }
checkAndAdjustSettings(*queue_settings, engine_args, mode > LoadingStrictnessLevel::CREATE, log); checkAndAdjustSettings(*queue_settings, mode > LoadingStrictnessLevel::CREATE);
object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true);
FormatFactory::instance().checkFormatName(configuration->format); FormatFactory::instance().checkFormatName(configuration->format);

View File

@ -13,8 +13,20 @@ def started_cluster():
main_configs=["config.xml"], main_configs=["config.xml"],
with_minio=True, with_minio=True,
) )
cluster.start() cluster.start()
# local disk requires its `path` directory to exist.
# the two paths below belong to `test1` and `test2` disks
node = cluster.instances["disks_app_test"]
for path in ["path1", "path2"]:
node.exec_in_container(
[
"bash",
"-c",
f"mkdir -p /var/lib/clickhouse/{path}",
]
)
yield cluster yield cluster
finally: finally:

View File

@ -13,5 +13,6 @@ Td1EnWQo [1,2,3,4]
XMbT XMbT
-- invalid sqid -- invalid sqid
[] []
-- bug 69450
-- alias -- alias
XMbT XMbT

View File

@ -25,5 +25,12 @@ SELECT sqidEncode(toNullable(materialize(1)), toLowCardinality(materialize(2)));
SELECT '-- invalid sqid'; SELECT '-- invalid sqid';
SELECT sqidDecode('invalid sqid'); SELECT sqidDecode('invalid sqid');
SELECT '-- bug 69450';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab (id String) ENGINE = MergeTree ORDER BY id;
INSERT INTO tab SELECT * FROM generateRandom() LIMIT 1000000;
SELECT sqidDecode(id) FROM tab FORMAT Null;
DROP TABLE tab;
SELECT '-- alias'; SELECT '-- alias';
SELECT sqid(1, 2); SELECT sqid(1, 2);

View File

@ -0,0 +1,10 @@
CREATE TABLE t (st FixedString(54)) ENGINE=MergeTree ORDER BY ();
INSERT INTO t VALUES
('abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRTUVWXYZ'),
('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'),
('IIIIIIIIII\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0');
WITH (SELECT groupConcat(',')(st) FROM t) AS a,
(SELECT groupConcat(',')(st :: String) FROM t) AS b
SELECT equals(a, b);