Merge branch 'master' into clickhouse-keeper

This commit is contained in:
mergify[bot] 2022-03-25 17:25:50 +00:00 committed by GitHub
commit 1dcbfe8739
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 73 additions and 27 deletions

View File

@ -43,6 +43,9 @@ public:
for (size_t i = 2; i < args.size() - 1; i += 2)
dst_array_types.push_back(args[i]);
// Type of the ELSE branch
dst_array_types.push_back(args.back());
return getLeastSupertype(dst_array_types);
}

View File

@ -1180,11 +1180,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
/// old instance of the storage. For example, AsynchronousMetrics may cause ATTACH to fail,
/// so we allow waiting here. If database_atomic_wait_for_drop_and_detach_synchronously is disabled
/// and old storage instance still exists it will throw exception.
bool throw_if_table_in_use = getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously;
if (throw_if_table_in_use)
database->checkDetachedTableNotInUse(create.uuid);
else
if (getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously)
database->waitDetachedTableNotInUse(create.uuid);
else
database->checkDetachedTableNotInUse(create.uuid);
}
StoragePtr res;

View File

@ -1,16 +1,22 @@
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <IO/WriteBuffer.h>
#include <IO/Operators.h>
#include <stack>
#include <Common/JSONBuilder.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ArrayJoinAction.h>
#include <stack>
#include <IO/Operators.h>
#include <IO/WriteBuffer.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Common/JSONBuilder.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
namespace DB
{
@ -388,6 +394,7 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt
static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSettings & settings)
{
settings.out << String(settings.offset, settings.indent_char) << "(" << step.getName() << ")\n";
size_t current_offset = settings.offset;
step.describePipeline(settings);
if (current_offset == settings.offset)

View File

@ -112,6 +112,9 @@ ReadFromMergeTree::ReadFromMergeTree(
if (enable_parallel_reading)
read_task_callback = context->getMergeTreeReadTaskCallback();
/// Add explicit description.
setStepDescription(data.getStorageID().getFullNameNotQuoted());
}
Pipe ReadFromMergeTree::readFromPool(

View File

@ -100,7 +100,8 @@ public:
bool enable_parallel_reading
);
String getName() const override { return "ReadFromMergeTree"; }
static constexpr auto name = "ReadFromMergeTree";
String getName() const override { return name; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;

View File

@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3)
sleep 1
$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK"
wait

View File

@ -9,7 +9,7 @@ Expression (Projection)
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_order_by)
SELECT
timestamp,
key
@ -21,7 +21,7 @@ Expression (Projection)
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_order_by)
SELECT
timestamp,
key
@ -35,7 +35,7 @@ Expression (Projection)
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_order_by)
SELECT
timestamp,
key

View File

@ -26,35 +26,35 @@ Expression (Projection)
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
optimize_aggregation_in_order
Expression ((Projection + Before ORDER BY))
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
Expression ((Projection + Before ORDER BY))
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
Expression ((Projection + Before ORDER BY))
Aggregating
Expression (Before GROUP BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree
ReadFromMergeTree (default.test_table)
second-index
1
1

View File

@ -1,4 +1,4 @@
ReadFromMergeTree
ReadFromMergeTree (default.test_index)
Indexes:
MinMax
Keys:
@ -32,6 +32,7 @@
Granules: 1/2
-----------------
"Node Type": "ReadFromMergeTree",
"Description": "default.test_index",
"Indexes": [
{
"Type": "MinMax",
@ -89,16 +90,16 @@
}
]
-----------------
ReadFromMergeTree
ReadFromMergeTree (default.test_index)
ReadType: InOrder
Parts: 1
Granules: 3
-----------------
ReadFromMergeTree
ReadFromMergeTree (default.test_index)
ReadType: InReverseOrder
Parts: 1
Granules: 3
ReadFromMergeTree
ReadFromMergeTree (default.idx)
Indexes:
PrimaryKey
Keys:

View File

@ -0,0 +1,20 @@
0 555555
1 10
2 555555
3 55
4 555555
5 555555
6 77
7 555555
8 555555
9 95
10 100
11 555555
12 555555
13 555555
14 555555
15 555555
16 555555
17 555555
18 555555
19 555555

View File

@ -0,0 +1,12 @@
SELECT "number", CASE "number"
WHEN 3 THEN 55
WHEN 6 THEN 77
WHEN 9 THEN 95
ELSE CASE
WHEN "number"=1 THEN 10
WHEN "number"=10 THEN 100
ELSE 555555
END
END AS "LONG_COL_0"
FROM `system`.numbers
LIMIT 20;