Merge branch 'master' into keeper-bench-replay-zookeeper-log

This commit is contained in:
Antonio Andelic 2024-05-22 09:28:49 +02:00
commit 056b95d7e5
23 changed files with 385 additions and 102 deletions

View File

@ -998,17 +998,170 @@ SELECT version()
Returns the build ID generated by a compiler for the running ClickHouse server binary.
If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
## blockNumber()
## blockNumber
Returns the sequence number of the data block where the row is located.
Returns a monotonically increasing sequence number of the [block](../../development/architecture.md#block) containing the row.
The returned block number is updated on a best-effort basis, i.e. it may not be fully accurate.
## rowNumberInBlock() {#rowNumberInBlock}
**Syntax**
```sql
blockNumber()
```
**Returned value**
- Sequence number of the data block where the row is located. [UInt64](../data-types/int-uint.md).
**Example**
Query:
```sql
SELECT blockNumber()
FROM
(
SELECT *
FROM system.numbers
LIMIT 10
) SETTINGS max_block_size = 2
```
Result:
```response
┌─blockNumber()─┐
│ 7 │
│ 7 │
└───────────────┘
┌─blockNumber()─┐
│ 8 │
│ 8 │
└───────────────┘
┌─blockNumber()─┐
│ 9 │
│ 9 │
└───────────────┘
┌─blockNumber()─┐
│ 10 │
│ 10 │
└───────────────┘
┌─blockNumber()─┐
│ 11 │
│ 11 │
└───────────────┘
```
## rowNumberInBlock {#rowNumberInBlock}
Returns for each [block](../../development/architecture.md#block) processed by `rowNumberInBlock` the number of the current row.
The returned number starts for each block at 0.
**Syntax**
```sql
rowNumberInBlock()
```
**Returned value**
- Ordinal number of the row in the data block starting from 0. [UInt64](../data-types/int-uint.md).
**Example**
Query:
```sql
SELECT rowNumberInBlock()
FROM
(
SELECT *
FROM system.numbers_mt
LIMIT 10
) SETTINGS max_block_size = 2
```
Result:
```response
┌─rowNumberInBlock()─┐
│ 0 │
│ 1 │
└────────────────────┘
┌─rowNumberInBlock()─┐
│ 0 │
│ 1 │
└────────────────────┘
┌─rowNumberInBlock()─┐
│ 0 │
│ 1 │
└────────────────────┘
┌─rowNumberInBlock()─┐
│ 0 │
│ 1 │
└────────────────────┘
┌─rowNumberInBlock()─┐
│ 0 │
│ 1 │
└────────────────────┘
```
## rowNumberInAllBlocks
Returns a unique row number for each row processed by `rowNumberInAllBlocks`. The returned numbers start at 0.
**Syntax**
```sql
rowNumberInAllBlocks()
```
**Returned value**
- Ordinal number of the row in the data block starting from 0. [UInt64](../data-types/int-uint.md).
**Example**
Query:
```sql
SELECT rowNumberInAllBlocks()
FROM
(
SELECT *
FROM system.numbers_mt
LIMIT 10
)
SETTINGS max_block_size = 2
```
Result:
```response
┌─rowNumberInAllBlocks()─┐
│ 0 │
│ 1 │
└────────────────────────┘
┌─rowNumberInAllBlocks()─┐
│ 4 │
│ 5 │
└────────────────────────┘
┌─rowNumberInAllBlocks()─┐
│ 2 │
│ 3 │
└────────────────────────┘
┌─rowNumberInAllBlocks()─┐
│ 6 │
│ 7 │
└────────────────────────┘
┌─rowNumberInAllBlocks()─┐
│ 8 │
│ 9 │
└────────────────────────┘
```
Returns the ordinal number of the row in the data block. Different data blocks are always recalculated.
## rowNumberInAllBlocks()
Returns the ordinal number of the row in the data block. This function only considers the affected data blocks.
## neighbor

View File

@ -92,7 +92,7 @@ void applySettingsQuirks(Settings & settings, LoggerPtr log)
void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
{
auto getCurrentValue = [&current_settings](const std::string_view name) -> Field
auto get_current_value = [&current_settings](const std::string_view name) -> Field
{
Field current_value;
bool has_current_value = current_settings.tryGet(name, current_value);
@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
return current_value;
};
UInt64 max_threads = getCurrentValue("max_threads").get<UInt64>();
UInt64 max_threads = get_current_value("max_threads").get<UInt64>();
UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores();
if (max_threads > max_threads_max_value)
{
@ -109,7 +109,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
current_settings.set("max_threads", max_threads_max_value);
}
constexpr UInt64 max_sane_block_rows_size = 4294967296; // 2^32
static constexpr UInt64 max_sane_block_rows_size = 4294967296; // 2^32
std::unordered_set<String> block_rows_settings{
"max_block_size",
"max_insert_block_size",
@ -120,13 +120,21 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
"input_format_parquet_max_block_size"};
for (auto const & setting : block_rows_settings)
{
auto block_size = getCurrentValue(setting).get<UInt64>();
if (block_size > max_sane_block_rows_size)
if (auto block_size = get_current_value(setting).get<UInt64>();
block_size > max_sane_block_rows_size)
{
if (log)
LOG_WARNING(log, "Sanity check: '{}' value is too high ({}). Reduced to {}", setting, block_size, max_sane_block_rows_size);
current_settings.set(setting, max_sane_block_rows_size);
}
}
if (auto max_block_size = get_current_value("max_block_size").get<UInt64>(); max_block_size == 0)
{
if (log)
LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE);
current_settings.set("max_block_size", DEFAULT_BLOCK_SIZE);
}
}
}

View File

@ -76,6 +76,20 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex
String replica_path = server_settings.default_replica_path;
String replica_name = server_settings.default_replica_name;
/// Check that replica path doesn't exist
Macros::MacroExpansionInfo info;
StorageID table_id = StorageID(create_query->getDatabase(), create_query->getTable(), create_query->uuid);
info.table_id = table_id;
info.expand_special_macros_only = false;
String zookeeper_path = context->getMacros()->expand(replica_path, info);
if (context->getZooKeeper()->exists(zookeeper_path))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Found existing ZooKeeper path {} while trying to convert table {} to replicated. Table will not be converted.",
zookeeper_path, backQuote(table_id.getFullTableName())
);
auto args = std::make_shared<ASTExpressionList>();
args->children.push_back(std::make_shared<ASTLiteral>(replica_path));
args->children.push_back(std::make_shared<ASTLiteral>(replica_name));

View File

@ -306,7 +306,6 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory)
bool /* skip_access_check */) -> ObjectStoragePtr
{
AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix);
std::string endpoint_string = endpoint.getEndpoint();
return createObjectStorage<AzureObjectStorage>(
ObjectStorageType::Azure, config, config_prefix, name,

View File

@ -300,21 +300,32 @@ void copyAzureBlobStorageFile(
if (size < settings->max_single_part_copy_size)
{
LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy blob sync {} -> {}", src_blob, dest_blob);
block_blob_client_dest.CopyFromUri(source_uri);
}
else
{
Azure::Storage::Blobs::StartBlobCopyOperation operation = block_blob_client_dest.StartCopyFromUri(source_uri);
// Wait for the operation to finish, checking for status every 100 second.
auto copy_response = operation.PollUntilDone(std::chrono::milliseconds(100));
auto properties_model = copy_response.Value;
if (properties_model.CopySource.HasValue())
{
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy failed");
}
auto copy_status = properties_model.CopyStatus;
auto copy_status_description = properties_model.CopyStatusDescription;
if (copy_status.HasValue() && copy_status.Value() == Azure::Storage::Blobs::Models::CopyStatus::Success)
{
LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copy of {} to {} finished", properties_model.CopySource.Value(), dest_blob);
}
else
{
if (copy_status.HasValue())
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} failed with status {} description {} (operation is done {})",
src_blob, dest_blob, copy_status.Value().ToString(), copy_status_description.Value(), operation.IsDone());
else
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy from {} to {} didn't complete with success status (operation is done {})", src_blob, dest_blob, operation.IsDone());
}
}
}
else

View File

@ -462,8 +462,8 @@ void ThreadStatus::initGlobalProfiler([[maybe_unused]] UInt64 global_profiler_re
{
#if !defined(SANITIZER) && !defined(__APPLE__)
/// profilers are useless without trace collector
auto global_context_ptr = global_context.lock();
if (!global_context_ptr || !global_context_ptr->hasTraceCollector())
auto context = Context::getGlobalContextInstance();
if (!context->hasTraceCollector())
return;
try

View File

@ -103,7 +103,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int QUERY_WAS_CANCELLED;
extern const int INCORRECT_DATA;
extern const int SYNTAX_ERROR;
extern const int SUPPORT_IS_DISABLED;
extern const int INCORRECT_QUERY;
@ -1256,34 +1255,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
}
}
// Here we check if our our projections contain force_optimize_projection_name
if (!settings.force_optimize_projection_name.value.empty())
{
bool found = false;
std::set<std::string> projections;
{
const auto & access_info = context->getQueryAccessInfo();
std::lock_guard lock(access_info.mutex);
projections = access_info.projections;
}
for (const auto &projection : projections)
{
// projection value has structure like: <db_name>.<table_name>.<projection_name>
// We need to get only the projection name
size_t last_dot_pos = projection.find_last_of('.');
std::string projection_name = (last_dot_pos != std::string::npos) ? projection.substr(last_dot_pos + 1) : projection;
if (settings.force_optimize_projection_name.value == projection_name)
{
found = true;
break;
}
}
if (!found)
throw Exception(ErrorCodes::INCORRECT_DATA, "Projection {} is specified in setting force_optimize_projection_name but not used",
settings.force_optimize_projection_name.value);
}
if (process_list_entry)
{

View File

@ -111,8 +111,11 @@ void optimizePrimaryKeyCondition(const Stack & stack);
void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes);
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections);
bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes);
/// Returns the name of used projection or nullopt if no projection is used.
std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections);
std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes);
bool addPlansForSets(QueryPlan & plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes);
/// Enable memory bound merging of aggregation states for remote queries

View File

@ -46,7 +46,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const
settings.optimize_projection = from.optimize_use_projections;
settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection;
settings.force_projection_name = from.force_optimize_projection_name;
settings.force_projection_name = settings.optimize_projection ? from.force_optimize_projection_name.value : "";
settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections;
return settings;

View File

@ -12,6 +12,7 @@ namespace DB
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int TOO_MANY_QUERY_PLAN_OPTIMIZATIONS;
extern const int PROJECTION_NOT_USED;
}
@ -106,7 +107,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query
void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes)
{
const size_t max_optimizations_to_apply = optimization_settings.max_optimizations_to_apply;
size_t num_applied_projection = 0;
std::unordered_set<String> applied_projection_names;
bool has_reading_from_mt = false;
Stack stack;
@ -159,9 +160,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
/// Projection optimization relies on PK optimization
if (optimization_settings.optimize_projection)
num_applied_projection
+= optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections);
{
auto applied_projection = optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections);
if (applied_projection)
applied_projection_names.insert(*applied_projection);
}
if (optimization_settings.aggregation_in_order)
optimizeAggregationInOrder(*frame.node, nodes);
@ -180,11 +183,11 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
if (optimization_settings.optimize_projection)
{
/// Projection optimization relies on PK optimization
if (optimizeUseNormalProjections(stack, nodes))
if (auto applied_projection = optimizeUseNormalProjections(stack, nodes))
{
++num_applied_projection;
applied_projection_names.insert(*applied_projection);
if (max_optimizations_to_apply && max_optimizations_to_apply < num_applied_projection)
if (max_optimizations_to_apply && max_optimizations_to_apply < applied_projection_names.size())
throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS,
"Too many projection optimizations applied to query plan. Current limit {}",
max_optimizations_to_apply);
@ -201,10 +204,16 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
stack.pop_back();
}
if (optimization_settings.force_use_projection && has_reading_from_mt && num_applied_projection == 0)
if (optimization_settings.force_use_projection && has_reading_from_mt && applied_projection_names.empty())
throw Exception(
ErrorCodes::PROJECTION_NOT_USED,
"No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1");
if (!optimization_settings.force_projection_name.empty() && has_reading_from_mt && !applied_projection_names.contains(optimization_settings.force_projection_name))
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Projection {} is specified in setting force_optimize_projection_name but not used",
optimization_settings.force_projection_name);
}
void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes)

View File

@ -552,28 +552,28 @@ static QueryPlan::Node * findReadingStep(QueryPlan::Node & node)
return nullptr;
}
bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections)
std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections)
{
if (node.children.size() != 1)
return false;
return {};
auto * aggregating = typeid_cast<AggregatingStep *>(node.step.get());
if (!aggregating)
return false;
return {};
if (!aggregating->canUseProjection())
return false;
return {};
QueryPlan::Node * reading_node = findReadingStep(*node.children.front());
if (!reading_node)
return false;
return {};
auto * reading = typeid_cast<ReadFromMergeTree *>(reading_node->step.get());
if (!reading)
return false;
return {};
if (!canUseProjectionForReadingStep(reading))
return false;
return {};
std::shared_ptr<PartitionIdToMaxBlock> max_added_blocks = getMaxAddedBlocks(reading);
@ -597,7 +597,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
if (ordinary_reading_marks == 0)
{
reading->setAnalyzedResult(std::move(ordinary_reading_select_result));
return false;
return {};
}
const auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges;
@ -631,15 +631,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
if (!best_candidate)
{
reading->setAnalyzedResult(std::move(ordinary_reading_select_result));
return false;
return {};
}
}
else
{
return false;
return {};
}
Context::QualifiedProjectionName projection_name;
chassert(best_candidate != nullptr);
QueryPlanStepPtr projection_reading;
@ -654,12 +653,6 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
Pipe pipe(std::make_shared<SourceFromSingleChunk>(std::move(candidates.minmax_projection->block)));
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
has_ordinary_parts = false;
projection_name = Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = candidates.minmax_projection->candidate.projection->name,
};
}
else
{
@ -691,12 +684,6 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
}
projection_name = Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
};
has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
if (has_ordinary_parts)
reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr));
@ -746,7 +733,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
node.children.push_back(&expr_or_filter_node);
}
return true;
return best_candidate->projection->name;
}
}

View File

@ -73,16 +73,16 @@ static bool hasAllRequiredColumns(const ProjectionDescription * projection, cons
}
bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
{
const auto & frame = stack.back();
auto * reading = typeid_cast<ReadFromMergeTree *>(frame.node->step.get());
if (!reading)
return false;
return {};
if (!canUseProjectionForReadingStep(reading))
return false;
return {};
auto iter = stack.rbegin();
while (std::next(iter) != stack.rend())
@ -96,7 +96,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
/// Dangling query plan node. This might be generated by StorageMerge.
if (iter->node->step.get() == reading)
return false;
return {};
const auto metadata = reading->getStorageMetadata();
const auto & projections = metadata->projections;
@ -107,7 +107,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
normal_projections.push_back(&projection);
if (normal_projections.empty())
return false;
return {};
ContextPtr context = reading->getContext();
auto it = std::find_if(normal_projections.begin(), normal_projections.end(), [&](const auto * projection)
@ -126,7 +126,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
{
auto & child = iter->node->children[iter->next_child - 1];
if (!query.build(*child))
return false;
return {};
if (query.dag)
query.dag->removeUnusedActions();
@ -146,7 +146,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
if (ordinary_reading_marks == 0)
{
reading->setAnalyzedResult(std::move(ordinary_reading_select_result));
return false;
return {};
}
const auto & parts_with_ranges = ordinary_reading_select_result->parts_with_ranges;
@ -185,7 +185,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
if (!best_candidate)
{
reading->setAnalyzedResult(std::move(ordinary_reading_select_result));
return false;
return {};
}
auto storage_snapshot = reading->getStorageSnapshot();
@ -283,8 +283,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
/// Here we remove last steps from stack to be able to optimize again.
/// In theory, read-in-order can be applied to projection.
stack.resize(iter.base() - stack.begin());
return true;
return best_candidate->projection->name;
}
}

View File

@ -110,6 +110,11 @@ def cluster():
main_configs=[path],
with_azurite=True,
)
cluster.add_instance(
"node3",
main_configs=[path],
with_azurite=True,
)
cluster.start()
yield cluster
@ -216,3 +221,37 @@ def test_backup_restore_on_merge_tree_different_container(cluster):
azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket")
azure_query(node2, f"DROP TABLE test_simple_merge_tree_different_bucket_restored")
def test_backup_restore_on_merge_tree_native_copy_async(cluster):
node3 = cluster.instances["node3"]
azure_query(
node3,
f"CREATE TABLE test_simple_merge_tree_async(key UInt64, data String) Engine = MergeTree() ORDER BY tuple() SETTINGS storage_policy='policy_azure_cache'",
)
azure_query(node3, f"INSERT INTO test_simple_merge_tree_async VALUES (1, 'a')")
backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_merge_tree_async_backup')"
print("BACKUP DEST", backup_destination)
azure_query(
node3,
f"BACKUP TABLE test_simple_merge_tree_async TO {backup_destination}",
settings={"azure_max_single_part_copy_size": 0},
)
assert node3.contains_in_log("using native copy")
azure_query(
node3,
f"RESTORE TABLE test_simple_merge_tree_async AS test_simple_merge_tree_async_restored FROM {backup_destination};",
settings={"azure_max_single_part_copy_size": 0},
)
assert (
azure_query(node3, f"SELECT * from test_simple_merge_tree_async_restored")
== "1\ta\n"
)
assert node3.contains_in_log("using native copy")
azure_query(node3, f"DROP TABLE test_simple_merge_tree_async")
azure_query(node3, f"DROP TABLE test_simple_merge_tree_async_restored")

View File

@ -19,4 +19,4 @@
<shard>01</shard>
</macros>
</clickhouse>
</clickhouse>

View File

@ -15,6 +15,6 @@
<shard>01</shard>
</macros>
<default_replica_path>/lol/kek/'/{uuid}</default_replica_path>
<default_replica_path>/clickhouse/'/{database}/{table}/{uuid}</default_replica_path>
</clickhouse>

View File

@ -6,7 +6,7 @@ cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance(
"ch1",
main_configs=[
"configs/config.d/clusters_unusual.xml",
"configs/config.d/clusters_zk_path.xml",
"configs/config.d/distributed_ddl.xml",
],
with_zookeeper=True,
@ -63,7 +63,7 @@ def check_tables():
)
.strip()
.startswith(
"ReplicatedReplacingMergeTree(\\'/lol/kek/\\\\\\'/{uuid}\\', \\'{replica}\\', D)"
"ReplicatedReplacingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', D)"
)
)
assert (
@ -73,7 +73,7 @@ def check_tables():
)
.strip()
.startswith(
"ReplicatedVersionedCollapsingMergeTree(\\'/lol/kek/\\\\\\'/{uuid}\\', \\'{replica}\\', Sign, Version)"
"ReplicatedVersionedCollapsingMergeTree(\\'/clickhouse/\\\\\\'/{database}/{table}/{uuid}\\', \\'{replica}\\', Sign, Version)"
)
)

View File

@ -0,0 +1,69 @@
import pytest
from test_modify_engine_on_restart.common import (
get_table_path,
set_convert_flags,
)
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance(
"ch1",
main_configs=[
"configs/config.d/clusters_zk_path.xml",
"configs/config.d/distributed_ddl.xml",
],
with_zookeeper=True,
macros={"replica": "node1"},
stay_alive=True,
)
database_name = "modify_engine_zk_path"
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def q(node, query):
return node.query(database=database_name, sql=query)
def test_modify_engine_fails_if_zk_path_exists(started_cluster):
ch1.query("CREATE DATABASE " + database_name)
q(
ch1,
"CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;",
)
uuid = q(
ch1,
f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'",
).strip("'[]\n")
q(
ch1,
f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/\\'/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;",
)
set_convert_flags(ch1, database_name, ["already_exists_1"])
table_data_path = get_table_path(ch1, "already_exists_1", database_name)
ch1.stop_clickhouse()
ch1.start_clickhouse(retry_start=False, expected_to_fail=True)
# Check if we can cancel convertation
ch1.exec_in_container(
[
"bash",
"-c",
f"rm {table_data_path}convert_to_replicated",
]
)
ch1.start_clickhouse()

View File

@ -1,4 +0,0 @@
SET send_logs_level = 'fatal';
SET max_block_size = 0;
SELECT number FROM system.numbers; -- { serverError 12 }

View File

@ -1,3 +1,5 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test
(
`id` UInt64,
@ -18,3 +20,16 @@ SELECT name FROM test GROUP BY name SETTINGS force_optimize_projection_name='pro
SELECT name FROM test GROUP BY name SETTINGS force_optimize_projection_name='non_existing_projection'; -- { serverError 117 }
SELECT name FROM test SETTINGS force_optimize_projection_name='projection_name'; -- { serverError 117 }
INSERT INTO test SELECT number, 'test' FROM numbers(1, 100) SETTINGS force_optimize_projection_name='projection_name';
SELECT 1 SETTINGS force_optimize_projection_name='projection_name';
SYSTEM FLUSH LOGS;
SELECT read_rows FROM system.query_log
WHERE current_database = currentDatabase()
AND query LIKE '%SELECT name FROM test%'
AND Settings['force_optimize_projection_name'] = 'projection_name'
AND type = 'ExceptionBeforeStart';
DROP TABLE test;

View File

@ -0,0 +1,7 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | grep -q "Sanity check: 'max_block_size' cannot be 0. Set to default value" && echo "OK" || echo "FAIL"