mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #61201 from ClickHouse/revert-61200-revert-60430-max-parallel-replicas-validate
Don't allow to set max_parallel_replicas to 0 as it doesn't make sense
This commit is contained in:
commit
3116650fa9
@ -207,11 +207,17 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
|
||||
max_entries = nested_pools.size();
|
||||
}
|
||||
else if (pool_mode == PoolMode::GET_ONE)
|
||||
{
|
||||
max_entries = 1;
|
||||
}
|
||||
else if (pool_mode == PoolMode::GET_MANY)
|
||||
{
|
||||
max_entries = settings.max_parallel_replicas;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown pool allocation mode");
|
||||
}
|
||||
|
||||
if (!priority_func)
|
||||
priority_func = makeGetPriorityFunc(settings);
|
||||
|
@ -82,7 +82,7 @@ std::vector<Connection *> HedgedConnectionsFactory::getManyConnections(PoolMode
|
||||
}
|
||||
case PoolMode::GET_MANY:
|
||||
{
|
||||
max_entries = max_parallel_replicas;
|
||||
max_entries = std::min(max_parallel_replicas, shuffled_pools.size());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -158,7 +158,7 @@ private:
|
||||
/// checking the number of requested replicas that are still in process).
|
||||
size_t requested_connections_count = 0;
|
||||
|
||||
const size_t max_parallel_replicas = 0;
|
||||
const size_t max_parallel_replicas = 1;
|
||||
const bool skip_unavailable_shards = false;
|
||||
};
|
||||
|
||||
|
@ -188,7 +188,7 @@ class IColumn;
|
||||
\
|
||||
M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \
|
||||
\
|
||||
M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
|
||||
M(NonZeroUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled. Should be always greater than 0", 0) \
|
||||
M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \
|
||||
M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \
|
||||
M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \
|
||||
|
@ -575,4 +575,40 @@ void SettingFieldCustom::readBinary(ReadBuffer & in)
|
||||
parseFromString(str);
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64::SettingFieldNonZeroUInt64(UInt64 x) : SettingFieldUInt64(x)
|
||||
{
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64::SettingFieldNonZeroUInt64(const DB::Field & f) : SettingFieldUInt64(f)
|
||||
{
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64 & SettingFieldNonZeroUInt64::operator=(UInt64 x)
|
||||
{
|
||||
SettingFieldUInt64::operator=(x);
|
||||
checkValueNonZero();
|
||||
return *this;
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64 & SettingFieldNonZeroUInt64::operator=(const DB::Field & f)
|
||||
{
|
||||
SettingFieldUInt64::operator=(f);
|
||||
checkValueNonZero();
|
||||
return *this;
|
||||
}
|
||||
|
||||
void SettingFieldNonZeroUInt64::parseFromString(const String & str)
|
||||
{
|
||||
SettingFieldUInt64::parseFromString(str);
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
void SettingFieldNonZeroUInt64::checkValueNonZero() const
|
||||
{
|
||||
if (value == 0)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "A setting's value has to be greater than 0");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -516,4 +516,19 @@ struct SettingFieldCustom
|
||||
void readBinary(ReadBuffer & in);
|
||||
};
|
||||
|
||||
struct SettingFieldNonZeroUInt64 : public SettingFieldUInt64
|
||||
{
|
||||
public:
|
||||
explicit SettingFieldNonZeroUInt64(UInt64 x = 1);
|
||||
explicit SettingFieldNonZeroUInt64(const Field & f);
|
||||
|
||||
SettingFieldNonZeroUInt64 & operator=(UInt64 x);
|
||||
SettingFieldNonZeroUInt64 & operator=(const Field & f);
|
||||
|
||||
void parseFromString(const String & str);
|
||||
|
||||
private:
|
||||
void checkValueNonZero() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -798,7 +798,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
!= parallel_replicas_before_analysis)
|
||||
{
|
||||
context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
context->setSetting("max_parallel_replicas", UInt64{0});
|
||||
context->setSetting("max_parallel_replicas", UInt64{1});
|
||||
need_analyze_again = true;
|
||||
}
|
||||
|
||||
@ -945,7 +945,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
|
||||
if (number_of_replicas_to_use <= 1)
|
||||
{
|
||||
context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
context->setSetting("max_parallel_replicas", UInt64{0});
|
||||
context->setSetting("max_parallel_replicas", UInt64{1});
|
||||
LOG_DEBUG(log, "Disabling parallel replicas because there aren't enough rows to read");
|
||||
return true;
|
||||
}
|
||||
|
@ -294,7 +294,7 @@ bool applyTrivialCountIfPossible(
|
||||
|
||||
/// The query could use trivial count if it didn't use parallel replicas, so let's disable it
|
||||
query_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
query_context->setSetting("max_parallel_replicas", UInt64{0});
|
||||
query_context->setSetting("max_parallel_replicas", UInt64{1});
|
||||
LOG_TRACE(getLogger("Planner"), "Disabling parallel replicas to be able to use a trivial count optimization");
|
||||
|
||||
}
|
||||
@ -777,7 +777,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
{
|
||||
planner_context->getMutableQueryContext()->setSetting(
|
||||
"allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{0});
|
||||
planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{1});
|
||||
LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read");
|
||||
}
|
||||
else if (number_of_replicas_to_use < settings.max_parallel_replicas)
|
||||
|
@ -25,7 +25,7 @@ function run_query_with_pure_parallel_replicas () {
|
||||
$CLICKHOUSE_CLIENT \
|
||||
--query "$2" \
|
||||
--query_id "${1}_disabled" \
|
||||
--max_parallel_replicas 0
|
||||
--max_parallel_replicas 1
|
||||
|
||||
$CLICKHOUSE_CLIENT \
|
||||
--query "$2" \
|
||||
@ -50,7 +50,7 @@ function run_query_with_custom_key_parallel_replicas () {
|
||||
$CLICKHOUSE_CLIENT \
|
||||
--query "$2" \
|
||||
--query_id "${1}_disabled" \
|
||||
--max_parallel_replicas 0
|
||||
--max_parallel_replicas 1
|
||||
|
||||
$CLICKHOUSE_CLIENT \
|
||||
--query "$2" \
|
||||
|
@ -0,0 +1,5 @@
|
||||
drop table if exists test_d;
|
||||
create table test_d engine=Distributed(test_cluster_two_shard_three_replicas_localhost, system, numbers);
|
||||
select * from test_d limit 10 settings max_parallel_replicas = 0, prefer_localhost_replica = 0; --{clientError BAD_ARGUMENTS}
|
||||
drop table test_d;
|
||||
|
Loading…
Reference in New Issue
Block a user